From 64b7c8549e8b277bf5524f4bc59608440cbba066 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 28 Oct 2015 00:02:57 -0700 Subject: [PATCH 01/31] Factor cross-task memory arbitration into own component. --- .../memory/CrossTaskMemoryArbitrator.scala | 141 ++++++++++++++++++ .../apache/spark/memory/MemoryManager.scala | 96 ++---------- .../spark/memory/MemoryManagerSuite.scala | 8 +- 3 files changed, 164 insertions(+), 81 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/memory/CrossTaskMemoryArbitrator.scala diff --git a/core/src/main/scala/org/apache/spark/memory/CrossTaskMemoryArbitrator.scala b/core/src/main/scala/org/apache/spark/memory/CrossTaskMemoryArbitrator.scala new file mode 100644 index 0000000000000..416bc49a57fdb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/memory/CrossTaskMemoryArbitrator.scala @@ -0,0 +1,141 @@ +/* + * 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.memory + +import javax.annotation.concurrent.{GuardedBy, ThreadSafe} + +import scala.collection.mutable + +import org.apache.spark.{Logging, SparkException} + +/** + * Implements policies and bookkeeping for sharing a fixed-size pool of memory between tasks. + * + * Tries to ensure that each task gets a reasonable share of memory, instead of some task ramping up + * to a large amount first and then causing others to spill to disk repeatedly. + * + * If there are N tasks, it ensures that each task 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 tasks and redo the calculations of 1 / 2N and 1 / N in waiting tasks whenever + * this set changes. This is all done by synchronizing access to mutable state and using wait() and + * notifyAll() to signal changes to callers. Prior to Spark 1.6, this arbitration of memory across + * tasks was performed by the ShuffleMemoryManager. + */ +@ThreadSafe +private[memory] class CrossTaskMemoryArbitrator( + memoryManager: MemoryManager, + maxMemory: () => Long, + poolName: String) extends Logging { + + /** + * Map from taskAttemptId -> memory consumption in bytes + */ + @GuardedBy("memoryManager") + private val memoryForTask = new mutable.HashMap[Long, Long]() + + /** + * Returns the memory consumption, in bytes, for the given task. + */ + def getMemoryUsageForTask(taskAttemptId: Long): Long = memoryManager.synchronized { + memoryForTask.getOrElse(taskAttemptId, 0L) + } + + /** + * Try to acquire up to `numBytes` of memory for the given task 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 + * task has a chance to ramp up to at least 1 / 2N of the total memory pool (where N is the # of + * active tasks) before it is forced to spill. This can happen if the number of tasks increase + * but an older task had a lot of memory already. + * + * @return the number of bytes granted to the task. + */ + def acquireMemory(numBytes: Long, taskAttemptId: Long): Long = memoryManager.synchronized { + assert(numBytes > 0, s"invalid number of bytes requested: $numBytes") + + // Add this task to the taskMemory map just so we can keep an accurate count of the number + // of active tasks, to let other tasks ramp down their memory in calls to `acquireMemory` + if (!memoryForTask.contains(taskAttemptId)) { + memoryForTask(taskAttemptId) = 0L + // This will later cause waiting tasks to wake up and check numTasks again + memoryManager.notifyAll() + } + + // Keep looping until we're either sure that we don't want to grant this request (because this + // task would have more than 1 / numActiveTasks of the memory) or we have enough free + // memory to give it (we always let each task get at least 1 / (2 * numActiveTasks)). + // TODO: simplify this to limit each task to its own slot + while (true) { + val numActiveTasks = memoryForTask.keys.size + val curMem = memoryForTask(taskAttemptId) + val freeMemory = maxMemory() - memoryForTask.values.sum + + // How much we can grant this task; don't let it grow to more than 1 / numActiveTasks; + // don't let it be negative + val maxToGrant = + math.min(numBytes, math.max(0, (maxMemory() / numActiveTasks) - curMem)) + // Only give it as much memory as is free, which might be none if it reached 1 / numTasks + val toGrant = math.min(maxToGrant, freeMemory) + + if (curMem < maxMemory() / (2 * numActiveTasks)) { + // We want to let each task get at least 1 / (2 * numActiveTasks) before blocking; + // if we can't give it this much now, wait for other tasks to free up memory + // (this happens if older tasks allocated lots of memory before N grew) + if (freeMemory >= math.min(maxToGrant, maxMemory() / (2 * numActiveTasks) - curMem)) { + memoryForTask(taskAttemptId) += toGrant + return toGrant + } else { + logInfo( + s"TID $taskAttemptId waiting for at least 1/2N of $poolName pool to be free") + memoryManager.wait() + } + } else { + memoryForTask(taskAttemptId) += toGrant + return toGrant + } + } + 0L // Never reached + } + + def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = memoryManager.synchronized { + val curMem = memoryForTask.getOrElse(taskAttemptId, 0L) + if (curMem < numBytes) { + throw new SparkException( + s"Internal error: release called on $numBytes bytes but task only has $curMem bytes " + + s"of memory from the $poolName pool") + } + if (memoryForTask.contains(taskAttemptId)) { + memoryForTask(taskAttemptId) -= numBytes + if (memoryForTask(taskAttemptId) <= 0) { + memoryForTask.remove(taskAttemptId) + } + } + memoryManager.notifyAll() // Notify waiters in acquireMemory() that memory has been freed + } + + /** + * Release all memory for the given task and mark it as inactive (e.g. when a task ends). + * @return the number of bytes freed. + */ + def releaseAllMemoryForTask(taskAttemptId: Long): Long = memoryManager.synchronized { + val numBytesToFree = getMemoryUsageForTask(taskAttemptId) + releaseMemory(numBytesToFree, taskAttemptId) + numBytesToFree + } +} diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 6c9a71c3855b0..ac15891e41622 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import com.google.common.annotations.VisibleForTesting -import org.apache.spark.{SparkException, TaskContext, SparkConf, Logging} +import org.apache.spark.{Logging, SparkConf, TaskContext} import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore} import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.memory.MemoryAllocator @@ -35,16 +35,6 @@ import org.apache.spark.unsafe.memory.MemoryAllocator * In this context, execution memory refers to that used for computation in shuffles, joins, * sorts and aggregations, while storage memory refers to that used for caching and propagating * internal data across the cluster. There exists one MemoryManager per JVM. - * - * The MemoryManager abstract base class itself implements policies for sharing execution memory - * between tasks; it tries to ensure that each task gets a reasonable share of memory, instead of - * some task ramping up to a large amount first and then causing others to spill to disk repeatedly. - * If there are N tasks, it ensures that each task 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 tasks and redo the calculations of 1 / 2N and 1 / N in waiting tasks whenever - * this set changes. This is all done by synchronizing access to mutable state and using wait() and - * notifyAll() to signal changes to callers. Prior to Spark 1.6, this arbitration of memory across - * tasks was performed by the ShuffleMemoryManager. */ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) extends Logging { @@ -62,8 +52,8 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte // Amount of execution/storage memory in use, accesses must be synchronized on `this` @GuardedBy("this") protected var _executionMemoryUsed: Long = 0 @GuardedBy("this") protected var _storageMemoryUsed: Long = 0 - // Map from taskAttemptId -> memory consumption in bytes - @GuardedBy("this") private val executionMemoryForTask = new mutable.HashMap[Long, Long]() + @GuardedBy("this") private[this] val onHeapExecutionMemoryArbitrator: CrossTaskMemoryArbitrator = + new CrossTaskMemoryArbitrator(this, maxExecutionMemory _, "on-heap execution") /** * Set the [[MemoryStore]] used by this manager to evict cached blocks. @@ -136,65 +126,22 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte */ private[memory] final def acquireExecutionMemory(numBytes: Long, taskAttemptId: Long): Long = synchronized { - assert(numBytes > 0, "invalid number of bytes requested: " + numBytes) - - // Add this task to the taskMemory map just so we can keep an accurate count of the number - // of active tasks, to let other tasks ramp down their memory in calls to tryToAcquire - if (!executionMemoryForTask.contains(taskAttemptId)) { - executionMemoryForTask(taskAttemptId) = 0L - // This will later cause waiting tasks to wake up and check numTasks again - notifyAll() - } - - // Once the cross-task memory allocation policy has decided to grant more memory to a task, - // this method is called in order to actually obtain that execution memory, potentially - // triggering eviction of storage memory: - def acquire(toGrant: Long): Long = synchronized { + val memoryGranted = onHeapExecutionMemoryArbitrator.acquireMemory(numBytes, taskAttemptId) + if (memoryGranted > 0L) { val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - val acquired = doAcquireExecutionMemory(toGrant, evictedBlocks) + val acquired = doAcquireExecutionMemory(memoryGranted, evictedBlocks) + assert(acquired <= memoryGranted) + onHeapExecutionMemoryArbitrator.releaseMemory(memoryGranted - acquired, taskAttemptId) // Register evicted blocks, if any, with the active task metrics Option(TaskContext.get()).foreach { tc => val metrics = tc.taskMetrics() val lastUpdatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) metrics.updatedBlocks = Some(lastUpdatedBlocks ++ evictedBlocks.toSeq) } - executionMemoryForTask(taskAttemptId) += acquired acquired + } else { + 0L } - - // Keep looping until we're either sure that we don't want to grant this request (because this - // task would have more than 1 / numActiveTasks of the memory) or we have enough free - // memory to give it (we always let each task get at least 1 / (2 * numActiveTasks)). - // TODO: simplify this to limit each task to its own slot - while (true) { - val numActiveTasks = executionMemoryForTask.keys.size - val curMem = executionMemoryForTask(taskAttemptId) - val freeMemory = maxExecutionMemory - executionMemoryForTask.values.sum - - // How much we can grant this task; don't let it grow to more than 1 / numActiveTasks; - // don't let it be negative - val maxToGrant = - math.min(numBytes, math.max(0, (maxExecutionMemory / numActiveTasks) - curMem)) - // Only give it as much memory as is free, which might be none if it reached 1 / numTasks - val toGrant = math.min(maxToGrant, freeMemory) - - if (curMem < maxExecutionMemory / (2 * numActiveTasks)) { - // We want to let each task get at least 1 / (2 * numActiveTasks) before blocking; - // if we can't give it this much now, wait for other tasks to free up memory - // (this happens if older tasks allocated lots of memory before N grew) - if ( - freeMemory >= math.min(maxToGrant, maxExecutionMemory / (2 * numActiveTasks) - curMem)) { - return acquire(toGrant) - } else { - logInfo( - s"TID $taskAttemptId waiting for at least 1/2N of execution memory pool to be free") - wait() - } - } else { - return acquire(toGrant) - } - } - 0L // Never reached } @VisibleForTesting @@ -213,19 +160,8 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte */ private[memory] final def releaseExecutionMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized { - val curMem = executionMemoryForTask.getOrElse(taskAttemptId, 0L) - if (curMem < numBytes) { - throw new SparkException( - s"Internal error: release called on $numBytes bytes but task only has $curMem") - } - if (executionMemoryForTask.contains(taskAttemptId)) { - executionMemoryForTask(taskAttemptId) -= numBytes - if (executionMemoryForTask(taskAttemptId) <= 0) { - executionMemoryForTask.remove(taskAttemptId) - } - releaseExecutionMemory(numBytes) - } - notifyAll() // Notify waiters in acquireExecutionMemory() that memory has been freed + onHeapExecutionMemoryArbitrator.releaseMemory(numBytes, taskAttemptId) + releaseExecutionMemory(numBytes) } /** @@ -233,9 +169,9 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte * @return the number of bytes freed. */ private[memory] def releaseAllExecutionMemoryForTask(taskAttemptId: Long): Long = synchronized { - val numBytesToFree = getExecutionMemoryUsageForTask(taskAttemptId) - releaseExecutionMemory(numBytesToFree, taskAttemptId) - numBytesToFree + val freedMemory = onHeapExecutionMemoryArbitrator.releaseAllMemoryForTask(taskAttemptId) + releaseExecutionMemory(freedMemory) + freedMemory } /** @@ -283,7 +219,7 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte * Returns the execution memory consumption, in bytes, for the given task. */ private[memory] def getExecutionMemoryUsageForTask(taskAttemptId: Long): Long = synchronized { - executionMemoryForTask.getOrElse(taskAttemptId, 0L) + onHeapExecutionMemoryArbitrator.getMemoryUsageForTask(taskAttemptId) } // -- Fields related to Tungsten managed memory ------------------------------------------------- diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 1265087743a98..ba9d85fe7b8e5 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -149,14 +149,20 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { assert(taskMemoryManager.acquireExecutionMemory(400L) === 400L) assert(taskMemoryManager.acquireExecutionMemory(400L) === 400L) assert(taskMemoryManager.acquireExecutionMemory(200L) === 100L) + assert(taskMemoryManager.getMemoryConsumptionForThisTask === 1000L) + assert(manager.executionMemoryUsed === 1000L) assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L) assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L) taskMemoryManager.releaseExecutionMemory(500L) + assert(taskMemoryManager.getMemoryConsumptionForThisTask === 500L) + assert(manager.executionMemoryUsed === 500L) assert(taskMemoryManager.acquireExecutionMemory(300L) === 300L) assert(taskMemoryManager.acquireExecutionMemory(300L) === 200L) - taskMemoryManager.cleanUpAllAllocatedMemory() + assert(taskMemoryManager.cleanUpAllAllocatedMemory() === 1000L) + assert(taskMemoryManager.getMemoryConsumptionForThisTask === 0L) + assert(manager.executionMemoryUsed === 0L) assert(taskMemoryManager.acquireExecutionMemory(1000L) === 1000L) assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L) } From 8396ed638cee5aaae64d000d53f4cb0c8db768b3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 28 Oct 2015 15:23:55 -0700 Subject: [PATCH 02/31] Hacky WIP approach at supporting fixed limit on off-heap execution memory. --- .../spark/memory/TaskMemoryManager.java | 46 ++++++++-- .../shuffle/sort/ShuffleExternalSorter.java | 16 ++-- .../apache/spark/memory/MemoryManager.scala | 92 ++++++++++++++----- .../spark/memory/StaticMemoryManager.scala | 8 +- .../spark/memory/UnifiedMemoryManager.scala | 12 +-- .../spark/util/collection/Spillable.scala | 4 +- .../spark/memory/TaskMemoryManagerSuite.java | 4 +- .../sort/UnsafeShuffleWriterSuite.java | 8 +- .../map/AbstractBytesToBytesMapSuite.java | 4 +- .../memory/GrantEverythingMemoryManager.scala | 14 ++- .../spark/memory/MemoryManagerSuite.scala | 60 ++++++------ .../memory/StaticMemoryManagerSuite.scala | 10 +- .../memory/UnifiedMemoryManagerSuite.scala | 6 +- .../spark/storage/BlockManagerSuite.scala | 2 +- 14 files changed, 189 insertions(+), 97 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 7b31c90dac666..f72efaded2b08 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -110,18 +110,33 @@ public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) { } /** - * Acquire N bytes of memory for execution, evicting cached blocks if necessary. + * Acquire N bytes of on-heap memory for execution, evicting cached blocks if necessary. * @return number of bytes successfully granted (<= N). */ - public long acquireExecutionMemory(long size) { - return memoryManager.acquireExecutionMemory(size, taskAttemptId); + public long acquireOnHeapExecutionMemory(long size) { + return memoryManager.acquireOnHeapExecutionMemory(size, taskAttemptId); } /** - * Release N bytes of execution memory. + * Acquire N bytes of off-heap memory for execution. + * @return number of bytes successfully granted (<= N). + */ + public long acquireOffHeapExecutionMemory(long size) { + return memoryManager.acquireOffHeapExecutionMemory(size, taskAttemptId); + } + + /** + * Release N bytes of on-heap execution memory. + */ + public void releaseOnHeapExecutionMemory(long size) { + memoryManager.releaseOnHeapExecutionMemory(size, taskAttemptId); + } + + /** + * Release N bytes of off-heap execution memory. */ - public void releaseExecutionMemory(long size) { - memoryManager.releaseExecutionMemory(size, taskAttemptId); + public void releaseOffHeapExecutionMemory(long size) { + memoryManager.releaseOffHeapExecutionMemory(size, taskAttemptId); } public long pageSizeBytes() { @@ -149,9 +164,18 @@ public MemoryBlock allocatePage(long size) { } allocatedPages.set(pageNumber); } - final long acquiredExecutionMemory = acquireExecutionMemory(size); + final long acquiredExecutionMemory; + if (memoryManager.tungstenMemoryIsAllocatedInHeap()) { + acquiredExecutionMemory = acquireOnHeapExecutionMemory(size); + } else { + acquiredExecutionMemory = acquireOffHeapExecutionMemory(size); + } if (acquiredExecutionMemory != size) { - releaseExecutionMemory(acquiredExecutionMemory); + if (memoryManager.tungstenMemoryIsAllocatedInHeap()) { + releaseOnHeapExecutionMemory(acquiredExecutionMemory); + } else { + releaseOffHeapExecutionMemory(acquiredExecutionMemory); + } synchronized (this) { allocatedPages.clear(pageNumber); } @@ -182,7 +206,11 @@ public void freePage(MemoryBlock page) { } long pageSize = page.size(); memoryManager.tungstenMemoryAllocator().free(page); - releaseExecutionMemory(pageSize); + if (memoryManager.tungstenMemoryIsAllocatedInHeap()) { + releaseOnHeapExecutionMemory(pageSize); + } else { + releaseOffHeapExecutionMemory(pageSize); + } } /** diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index f43236f41ae7b..4a854058e5f51 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -136,9 +136,9 @@ public ShuffleExternalSorter( private void initializeForWriting() throws IOException { // TODO: move this sizing calculation logic into a static method of sorter: final long memoryRequested = initialSize * 8L; - final long memoryAcquired = taskMemoryManager.acquireExecutionMemory(memoryRequested); + final long memoryAcquired = taskMemoryManager.acquireOnHeapExecutionMemory(memoryRequested); if (memoryAcquired != memoryRequested) { - taskMemoryManager.releaseExecutionMemory(memoryAcquired); + taskMemoryManager.releaseOnHeapExecutionMemory(memoryAcquired); throw new IOException("Could not acquire " + memoryRequested + " bytes of memory"); } @@ -278,7 +278,7 @@ void spill() throws IOException { writeSortedFile(false); final long inMemSorterMemoryUsage = inMemSorter.getMemoryUsage(); inMemSorter = null; - taskMemoryManager.releaseExecutionMemory(inMemSorterMemoryUsage); + taskMemoryManager.releaseOnHeapExecutionMemory(inMemSorterMemoryUsage); final long spillSize = freeMemory(); taskContext.taskMetrics().incMemoryBytesSpilled(spillSize); @@ -318,7 +318,7 @@ private long freeMemory() { if (inMemSorter != null) { long sorterMemoryUsage = inMemSorter.getMemoryUsage(); inMemSorter = null; - taskMemoryManager.releaseExecutionMemory(sorterMemoryUsage); + taskMemoryManager.releaseOnHeapExecutionMemory(sorterMemoryUsage); } allocatedPages.clear(); currentPage = null; @@ -340,7 +340,7 @@ public void cleanupResources() { if (inMemSorter != null) { long sorterMemoryUsage = inMemSorter.getMemoryUsage(); inMemSorter = null; - taskMemoryManager.releaseExecutionMemory(sorterMemoryUsage); + taskMemoryManager.releaseOnHeapExecutionMemory(sorterMemoryUsage); } } @@ -355,13 +355,13 @@ private void growPointerArrayIfNecessary() throws IOException { logger.debug("Attempting to expand sort pointer array"); final long oldPointerArrayMemoryUsage = inMemSorter.getMemoryUsage(); final long memoryToGrowPointerArray = oldPointerArrayMemoryUsage * 2; - final long memoryAcquired = taskMemoryManager.acquireExecutionMemory(memoryToGrowPointerArray); + final long memoryAcquired = taskMemoryManager.acquireOnHeapExecutionMemory(memoryToGrowPointerArray); if (memoryAcquired < memoryToGrowPointerArray) { - taskMemoryManager.releaseExecutionMemory(memoryAcquired); + taskMemoryManager.releaseOnHeapExecutionMemory(memoryAcquired); spill(); } else { inMemSorter.expandPointerArray(); - taskMemoryManager.releaseExecutionMemory(oldPointerArrayMemoryUsage); + taskMemoryManager.releaseOnHeapExecutionMemory(oldPointerArrayMemoryUsage); } } } diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index ac15891e41622..36419c7fafd8e 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -50,10 +50,13 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte } // Amount of execution/storage memory in use, accesses must be synchronized on `this` - @GuardedBy("this") protected var _executionMemoryUsed: Long = 0 + @GuardedBy("this") protected var _onHeapExecutionMemoryUsed: Long = 0 + @GuardedBy("this") protected var _offHeapExecutionMemoryUsed: Long = 0 @GuardedBy("this") protected var _storageMemoryUsed: Long = 0 @GuardedBy("this") private[this] val onHeapExecutionMemoryArbitrator: CrossTaskMemoryArbitrator = - new CrossTaskMemoryArbitrator(this, maxExecutionMemory _, "on-heap execution") + new CrossTaskMemoryArbitrator(this, maxOnHeapExecutionMemory _, "on-heap execution") + @GuardedBy("this") private[this] val offHeapExecutionMemoryArbitrator: CrossTaskMemoryArbitrator = + new CrossTaskMemoryArbitrator(this, maxOffHeapExecutionMemory _, "off-heap execution") /** * Set the [[MemoryStore]] used by this manager to evict cached blocks. @@ -64,9 +67,14 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte } /** - * Total available memory for execution, in bytes. + * Total on-heap memory available for execution, in bytes. */ - def maxExecutionMemory: Long + def maxOnHeapExecutionMemory: Long + + /** + * Total off-heap memory available for execution, in bytes. + */ + final def maxOffHeapExecutionMemory: Long = conf.getSizeAsBytes("spark.memory.offHeapSize", 0) /** * Total available memory for storage, in bytes. @@ -113,8 +121,8 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long /** - * Try to acquire up to `numBytes` of execution memory for the current task and return the number - * of bytes obtained, or 0 if none can be allocated. + * Try to acquire up to `numBytes` of on-heap execution memory for the current task 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 * task has a chance to ramp up to at least 1 / 2N of the total memory pool (where N is the # of @@ -125,7 +133,7 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte * that control global sharing of memory between execution and storage. */ private[memory] - final def acquireExecutionMemory(numBytes: Long, taskAttemptId: Long): Long = synchronized { + final def acquireOnHeapExecutionMemory(numBytes: Long, taskAttemptId: Long): Long = synchronized { val memoryGranted = onHeapExecutionMemoryArbitrator.acquireMemory(numBytes, taskAttemptId) if (memoryGranted > 0L) { val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -144,24 +152,64 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte } } + /** + * Try to acquire up to `numBytes` of off-heap execution memory for the current task 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 + * task has a chance to ramp up to at least 1 / 2N of the total memory pool (where N is the # of + * active tasks) before it is forced to spill. This can happen if the number of tasks increase + * but an older task had a lot of memory already. + */ + private[memory] + def acquireOffHeapExecutionMemory( + numBytes: Long, + taskAttemptId: Long): Long = synchronized { + val memoryGranted = offHeapExecutionMemoryArbitrator.acquireMemory(numBytes, taskAttemptId) + _offHeapExecutionMemoryUsed += memoryGranted + memoryGranted + } + + @VisibleForTesting + private[memory] def releaseOnHeapExecutionMemory(numBytes: Long): Unit = synchronized { + if (numBytes > _onHeapExecutionMemoryUsed) { + logWarning(s"Attempted to release $numBytes bytes of on-heap execution " + + s"memory when we only have ${_onHeapExecutionMemoryUsed} bytes") + _onHeapExecutionMemoryUsed = 0 + } else { + _onHeapExecutionMemoryUsed -= numBytes + } + } + @VisibleForTesting - private[memory] def releaseExecutionMemory(numBytes: Long): Unit = synchronized { - if (numBytes > _executionMemoryUsed) { - logWarning(s"Attempted to release $numBytes bytes of execution " + - s"memory when we only have ${_executionMemoryUsed} bytes") - _executionMemoryUsed = 0 + private[memory] def releaseOffHeapExecutionMemory(numBytes: Long): Unit = synchronized { + if (numBytes > _offHeapExecutionMemoryUsed) { + logWarning(s"Attempted to release $numBytes bytes of off-heap execution " + + s"memory when we only have ${_offHeapExecutionMemoryUsed} bytes") + _offHeapExecutionMemoryUsed = 0 } else { - _executionMemoryUsed -= numBytes + _offHeapExecutionMemoryUsed -= numBytes } } /** - * Release numBytes of execution memory belonging to the given task. + * Release numBytes of on-heap execution memory belonging to the given task. */ private[memory] - final def releaseExecutionMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized { + final def releaseOnHeapExecutionMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized { onHeapExecutionMemoryArbitrator.releaseMemory(numBytes, taskAttemptId) - releaseExecutionMemory(numBytes) + releaseOnHeapExecutionMemory(numBytes) + } + + /** + * Release numBytes of off-heap execution memory belonging to the given task. + */ + private[memory] + final def releaseOffHeapExecutionMemory( + numBytes: Long, + taskAttemptId: Long): Unit = synchronized { + offHeapExecutionMemoryArbitrator.releaseMemory(numBytes, taskAttemptId) + releaseOffHeapExecutionMemory(numBytes) } /** @@ -169,9 +217,11 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte * @return the number of bytes freed. */ private[memory] def releaseAllExecutionMemoryForTask(taskAttemptId: Long): Long = synchronized { - val freedMemory = onHeapExecutionMemoryArbitrator.releaseAllMemoryForTask(taskAttemptId) - releaseExecutionMemory(freedMemory) - freedMemory + val freedOnHeap = onHeapExecutionMemoryArbitrator.releaseAllMemoryForTask(taskAttemptId) + releaseOnHeapExecutionMemory(freedOnHeap) + val freedOffHeap = offHeapExecutionMemoryArbitrator.releaseAllMemoryForTask(taskAttemptId) + releaseOffHeapExecutionMemory(freedOffHeap) + freedOnHeap + freedOffHeap } /** @@ -205,7 +255,7 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte * Execution memory currently in use, in bytes. */ final def executionMemoryUsed: Long = synchronized { - _executionMemoryUsed + _onHeapExecutionMemoryUsed } /** @@ -237,7 +287,7 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte val cores = if (numCores > 0) numCores else Runtime.getRuntime.availableProcessors() // Because of rounding to next power of 2, we may have safetyFactor as 8 in worst case val safetyFactor = 16 - val size = ByteArrayMethods.nextPowerOf2(maxExecutionMemory / cores / safetyFactor) + val size = ByteArrayMethods.nextPowerOf2(maxOnHeapExecutionMemory / cores / safetyFactor) val default = math.min(maxPageSize, math.max(minPageSize, size)) conf.getSizeAsBytes("spark.buffer.pageSize", default) } diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala index 9c2c2e90a2282..c1e22f8551d50 100644 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala @@ -32,7 +32,7 @@ import org.apache.spark.storage.{BlockId, BlockStatus} */ private[spark] class StaticMemoryManager( conf: SparkConf, - override val maxExecutionMemory: Long, + override val maxOnHeapExecutionMemory: Long, override val maxStorageMemory: Long, numCores: Int) extends MemoryManager(conf, numCores) { @@ -58,9 +58,9 @@ private[spark] class StaticMemoryManager( numBytes: Long, evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long = synchronized { assert(numBytes >= 0) - assert(_executionMemoryUsed <= maxExecutionMemory) - val bytesToGrant = math.min(numBytes, maxExecutionMemory - _executionMemoryUsed) - _executionMemoryUsed += bytesToGrant + assert(_onHeapExecutionMemoryUsed <= maxOnHeapExecutionMemory) + val bytesToGrant = math.min(numBytes, maxOnHeapExecutionMemory - _onHeapExecutionMemoryUsed) + _onHeapExecutionMemoryUsed += bytesToGrant bytesToGrant } diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index a3093030a0f93..15b7e53ac2519 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -66,17 +66,17 @@ private[spark] class UnifiedMemoryManager( * Total amount of memory, in bytes, not currently occupied by either execution or storage. */ private def totalFreeMemory: Long = synchronized { - assert(_executionMemoryUsed <= maxMemory) + assert(_onHeapExecutionMemoryUsed <= maxMemory) assert(_storageMemoryUsed <= maxMemory) - assert(_executionMemoryUsed + _storageMemoryUsed <= maxMemory) - maxMemory - _executionMemoryUsed - _storageMemoryUsed + assert(_onHeapExecutionMemoryUsed + _storageMemoryUsed <= maxMemory) + maxMemory - _onHeapExecutionMemoryUsed - _storageMemoryUsed } /** * Total available memory for execution, in bytes. * In this model, this is equivalent to the amount of memory not occupied by storage. */ - override def maxExecutionMemory: Long = synchronized { + override def maxOnHeapExecutionMemory: Long = synchronized { maxMemory - _storageMemoryUsed } @@ -85,7 +85,7 @@ private[spark] class UnifiedMemoryManager( * In this model, this is equivalent to the amount of memory not occupied by execution. */ override def maxStorageMemory: Long = synchronized { - maxMemory - _executionMemoryUsed + maxMemory - _onHeapExecutionMemoryUsed } /** @@ -108,7 +108,7 @@ private[spark] class UnifiedMemoryManager( memoryStore.ensureFreeSpace(spaceToEnsure, evictedBlocks) } val bytesToGrant = math.min(numBytes, totalFreeMemory) - _executionMemoryUsed += bytesToGrant + _onHeapExecutionMemoryUsed += bytesToGrant bytesToGrant } diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index a76891acf0baf..4a1b9362df16e 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -78,7 +78,7 @@ private[spark] trait Spillable[C] extends Logging { if (elementsRead % 32 == 0 && currentMemory >= myMemoryThreshold) { // Claim up to double our current memory from the shuffle memory pool val amountToRequest = 2 * currentMemory - myMemoryThreshold - val granted = taskMemoryManager.acquireExecutionMemory(amountToRequest) + val granted = taskMemoryManager.acquireOnHeapExecutionMemory(amountToRequest) myMemoryThreshold += granted // If 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 @@ -107,7 +107,7 @@ private[spark] trait Spillable[C] extends Logging { */ def releaseMemory(): Unit = { // The amount we requested does not include the initial memory tracking threshold - taskMemoryManager.releaseExecutionMemory(myMemoryThreshold - initialMemoryThreshold) + taskMemoryManager.releaseOnHeapExecutionMemory(myMemoryThreshold - initialMemoryThreshold) myMemoryThreshold = initialMemoryThreshold } diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index f381db0c62653..a3be212e3bc2b 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -36,7 +36,9 @@ public void leakedPageMemoryIsDetected() { @Test public void encodePageNumberAndOffsetOffHeap() { final TaskMemoryManager manager = new TaskMemoryManager( - new GrantEverythingMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "true")), 0); + new GrantEverythingMemoryManager( + new SparkConf().set("spark.unsafe.offHeap", "true") + .set("spark.memory.offHeapSize", "1024b")), 0); final MemoryBlock dataPage = manager.allocatePage(256); // In off-heap mode, an offset is an absolute address that may require more than 51 bits to // encode. This test exercises that corner-case: diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index d65926949c036..304221af41917 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -403,7 +403,7 @@ public void writeEnoughDataToTriggerSpill() throws Exception { .doCallRealMethod() // allocate initial data page .doReturn(0L) // deny request to allocate new page .doCallRealMethod() // grant new sort buffer and data page - .when(taskMemoryManager).acquireExecutionMemory(anyLong()); + .when(taskMemoryManager).acquireOnHeapExecutionMemory(anyLong()); final UnsafeShuffleWriter writer = createWriter(false); final ArrayList> dataToWrite = new ArrayList>(); final byte[] bigByteArray = new byte[PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES / 128]; @@ -411,7 +411,7 @@ public void writeEnoughDataToTriggerSpill() throws Exception { dataToWrite.add(new Tuple2(i, bigByteArray)); } writer.write(dataToWrite.iterator()); - verify(taskMemoryManager, times(5)).acquireExecutionMemory(anyLong()); + verify(taskMemoryManager, times(5)).acquireOnHeapExecutionMemory(anyLong()); assertEquals(2, spillFilesCreated.size()); writer.stop(true); readRecordsFromFile(); @@ -431,14 +431,14 @@ public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exce .doCallRealMethod() // allocate initial data page .doReturn(0L) // deny request to allocate new page .doCallRealMethod() // grant new sort buffer and data page - .when(taskMemoryManager).acquireExecutionMemory(anyLong()); + .when(taskMemoryManager).acquireOnHeapExecutionMemory(anyLong()); final UnsafeShuffleWriter writer = createWriter(false); final ArrayList> dataToWrite = new ArrayList<>(); for (int i = 0; i < UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE; i++) { dataToWrite.add(new Tuple2(i, i)); } writer.write(dataToWrite.iterator()); - verify(taskMemoryManager, times(5)).acquireExecutionMemory(anyLong()); + verify(taskMemoryManager, times(5)).acquireOnHeapExecutionMemory(anyLong()); assertEquals(2, spillFilesCreated.size()); writer.stop(true); readRecordsFromFile(); diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java index 6e52496cf933b..22ab6d1615681 100644 --- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java +++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -45,7 +45,9 @@ public abstract class AbstractBytesToBytesMapSuite { public void setup() { memoryManager = new GrantEverythingMemoryManager( - new SparkConf().set("spark.unsafe.offHeap", "" + useOffHeapMemoryAllocator())); + new SparkConf() + .set("spark.unsafe.offHeap", "" + useOffHeapMemoryAllocator()) + .set("spark.memory.offHeapSize", "256mb")); taskMemoryManager = new TaskMemoryManager(memoryManager, 0); } diff --git a/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala index fe102d8aeb2a5..b9203b2604d67 100644 --- a/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala +++ b/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala @@ -30,7 +30,17 @@ class GrantEverythingMemoryManager(conf: SparkConf) extends MemoryManager(conf, oom = false 0 } else { - _executionMemoryUsed += numBytes // To suppress warnings when freeing unallocated memory + _onHeapExecutionMemoryUsed += numBytes // To suppress warnings when freeing unallocated memory + numBytes + } + } + // TODO(josh): this code is becoming very confusing; refactoring needed + override def acquireOffHeapExecutionMemory(numBytes: Long, taskId: Long): Long = synchronized { + if (oom) { + oom = false + 0 + } else { + _offHeapExecutionMemoryUsed += numBytes // To suppress warnings when freeing unallocated memory numBytes } } @@ -43,7 +53,7 @@ class GrantEverythingMemoryManager(conf: SparkConf) extends MemoryManager(conf, numBytes: Long, evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = true override def releaseStorageMemory(numBytes: Long): Unit = { } - override def maxExecutionMemory: Long = Long.MaxValue + override def maxOnHeapExecutionMemory: Long = Long.MaxValue override def maxStorageMemory: Long = Long.MaxValue private var oom = false diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index ba9d85fe7b8e5..a1cc3b8595fc9 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -145,26 +145,26 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val manager = createMemoryManager(1000L) val taskMemoryManager = new TaskMemoryManager(manager, 0) - assert(taskMemoryManager.acquireExecutionMemory(100L) === 100L) - assert(taskMemoryManager.acquireExecutionMemory(400L) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(400L) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(200L) === 100L) + assert(taskMemoryManager.acquireOnHeapExecutionMemory(100L) === 100L) + assert(taskMemoryManager.acquireOnHeapExecutionMemory(400L) === 400L) + assert(taskMemoryManager.acquireOnHeapExecutionMemory(400L) === 400L) + assert(taskMemoryManager.acquireOnHeapExecutionMemory(200L) === 100L) assert(taskMemoryManager.getMemoryConsumptionForThisTask === 1000L) assert(manager.executionMemoryUsed === 1000L) - assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L) - assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L) + assert(taskMemoryManager.acquireOnHeapExecutionMemory(100L) === 0L) + assert(taskMemoryManager.acquireOnHeapExecutionMemory(100L) === 0L) - taskMemoryManager.releaseExecutionMemory(500L) + taskMemoryManager.releaseOnHeapExecutionMemory(500L) assert(taskMemoryManager.getMemoryConsumptionForThisTask === 500L) assert(manager.executionMemoryUsed === 500L) - assert(taskMemoryManager.acquireExecutionMemory(300L) === 300L) - assert(taskMemoryManager.acquireExecutionMemory(300L) === 200L) + assert(taskMemoryManager.acquireOnHeapExecutionMemory(300L) === 300L) + assert(taskMemoryManager.acquireOnHeapExecutionMemory(300L) === 200L) assert(taskMemoryManager.cleanUpAllAllocatedMemory() === 1000L) assert(taskMemoryManager.getMemoryConsumptionForThisTask === 0L) assert(manager.executionMemoryUsed === 0L) - assert(taskMemoryManager.acquireExecutionMemory(1000L) === 1000L) - assert(taskMemoryManager.acquireExecutionMemory(100L) === 0L) + assert(taskMemoryManager.acquireOnHeapExecutionMemory(1000L) === 1000L) + assert(taskMemoryManager.acquireOnHeapExecutionMemory(100L) === 0L) } test("two tasks requesting full execution memory") { @@ -174,15 +174,15 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val futureTimeout: Duration = 20.seconds // Have both tasks request 500 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L) } + val t1Result1 = Future { t1MemManager.acquireOnHeapExecutionMemory(500L) } + val t2Result1 = Future { t2MemManager.acquireOnHeapExecutionMemory(500L) } assert(Await.result(t1Result1, futureTimeout) === 500L) assert(Await.result(t2Result1, futureTimeout) === 500L) // Have both tasks each request 500 bytes more; both should immediately return 0 as they are // both now at 1 / N - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L) } + val t1Result2 = Future { t1MemManager.acquireOnHeapExecutionMemory(500L) } + val t2Result2 = Future { t2MemManager.acquireOnHeapExecutionMemory(500L) } assert(Await.result(t1Result2, 200.millis) === 0L) assert(Await.result(t2Result2, 200.millis) === 0L) } @@ -194,15 +194,15 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val futureTimeout: Duration = 20.seconds // Have both tasks request 250 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L) } + val t1Result1 = Future { t1MemManager.acquireOnHeapExecutionMemory(250L) } + val t2Result1 = Future { t2MemManager.acquireOnHeapExecutionMemory(250L) } assert(Await.result(t1Result1, futureTimeout) === 250L) assert(Await.result(t2Result1, futureTimeout) === 250L) // Have both tasks each request 500 bytes more. // We should only grant 250 bytes to each of them on this second request - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L) } + val t1Result2 = Future { t1MemManager.acquireOnHeapExecutionMemory(500L) } + val t2Result2 = Future { t2MemManager.acquireOnHeapExecutionMemory(500L) } assert(Await.result(t1Result2, futureTimeout) === 250L) assert(Await.result(t2Result2, futureTimeout) === 250L) } @@ -214,17 +214,17 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L) } + val t1Result1 = Future { t1MemManager.acquireOnHeapExecutionMemory(1000L) } assert(Await.result(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L) } + val t2Result1 = Future { t2MemManager.acquireOnHeapExecutionMemory(250L) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) - t1MemManager.releaseExecutionMemory(250L) + t1MemManager.releaseOnHeapExecutionMemory(250L) // The memory freed from t1 should now be granted to t2. assert(Await.result(t2Result1, futureTimeout) === 250L) // Further requests by t2 should be denied immediately because it now has 1 / 2N of the memory. - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L) } + val t2Result2 = Future { t2MemManager.acquireOnHeapExecutionMemory(100L) } assert(Await.result(t2Result2, 200.millis) === 0L) } @@ -235,18 +235,18 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L) } + val t1Result1 = Future { t1MemManager.acquireOnHeapExecutionMemory(1000L) } assert(Await.result(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L) } + val t2Result1 = Future { t2MemManager.acquireOnHeapExecutionMemory(500L) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) // t1 releases all of its memory, so t2 should be able to grab all of the memory t1MemManager.cleanUpAllAllocatedMemory() assert(Await.result(t2Result1, futureTimeout) === 500L) - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L) } + val t2Result2 = Future { t2MemManager.acquireOnHeapExecutionMemory(500L) } assert(Await.result(t2Result2, futureTimeout) === 500L) - val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L) } + val t2Result3 = Future { t2MemManager.acquireOnHeapExecutionMemory(500L) } assert(Await.result(t2Result3, 200.millis) === 0L) } @@ -257,13 +257,13 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val t2MemManager = new TaskMemoryManager(memoryManager, 2) val futureTimeout: Duration = 20.seconds - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L) } + val t1Result1 = Future { t1MemManager.acquireOnHeapExecutionMemory(700L) } assert(Await.result(t1Result1, futureTimeout) === 700L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L) } + val t2Result1 = Future { t2MemManager.acquireOnHeapExecutionMemory(300L) } assert(Await.result(t2Result1, futureTimeout) === 300L) - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L) } + val t1Result2 = Future { t1MemManager.acquireOnHeapExecutionMemory(300L) } assert(Await.result(t1Result2, 200.millis) === 0L) } } diff --git a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala index 885c450d6d4f5..4748edbc5baf4 100644 --- a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala @@ -36,7 +36,7 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { maxExecutionMem: Long, maxStorageMem: Long): (StaticMemoryManager, MemoryStore) = { val mm = new StaticMemoryManager( - conf, maxExecutionMemory = maxExecutionMem, maxStorageMemory = maxStorageMem, numCores = 1) + conf, maxOnHeapExecutionMemory = maxExecutionMem, maxStorageMemory = maxStorageMem, numCores = 1) val ms = makeMemoryStore(mm) (mm, ms) } @@ -44,7 +44,7 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { override protected def createMemoryManager(maxMemory: Long): MemoryManager = { new StaticMemoryManager( conf, - maxExecutionMemory = maxMemory, + maxOnHeapExecutionMemory = maxMemory, maxStorageMemory = 0, numCores = 1) } @@ -61,13 +61,13 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { assert(mm.executionMemoryUsed === maxExecutionMem) assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 0L) assert(mm.executionMemoryUsed === maxExecutionMem) - mm.releaseExecutionMemory(800L) + mm.releaseOnHeapExecutionMemory(800L) assert(mm.executionMemoryUsed === 200L) // Acquire after release assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 1L) assert(mm.executionMemoryUsed === 201L) // Release beyond what was acquired - mm.releaseExecutionMemory(maxExecutionMem) + mm.releaseOnHeapExecutionMemory(maxExecutionMem) assert(mm.executionMemoryUsed === 0L) } @@ -128,7 +128,7 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { assert(mm.storageMemoryUsed === 50L) assert(mm.executionMemoryUsed === 200L) // Only execution memory should be released - mm.releaseExecutionMemory(133L) + mm.releaseOnHeapExecutionMemory(133L) assert(mm.storageMemoryUsed === 50L) assert(mm.executionMemoryUsed === 67L) // Only storage memory should be released diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 0c97f2bd89651..1c45857eda442 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -68,13 +68,13 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes assert(mm.executionMemoryUsed === maxMemory) assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 0L) assert(mm.executionMemoryUsed === maxMemory) - mm.releaseExecutionMemory(800L) + mm.releaseOnHeapExecutionMemory(800L) assert(mm.executionMemoryUsed === 200L) // Acquire after release assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 1L) assert(mm.executionMemoryUsed === 201L) // Release beyond what was acquired - mm.releaseExecutionMemory(maxMemory) + mm.releaseOnHeapExecutionMemory(maxMemory) assert(mm.executionMemoryUsed === 0L) } @@ -189,7 +189,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes assert(mm.executionMemoryUsed === 800L) assert(mm.storageMemoryUsed === 100L) assertEnsureFreeSpaceCalled(ms, 250L) - mm.releaseExecutionMemory(maxMemory) + mm.releaseOnHeapExecutionMemory(maxMemory) mm.releaseStorageMemory(maxMemory) // Acquire some execution memory again, but this time keep it within the execution region assert(mm.doAcquireExecutionMemory(200L, evictedBlocks) === 200L) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index d49015afcd594..53991d8a1aede 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -825,7 +825,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val transfer = new NettyBlockTransferService(conf, securityMgr, numCores = 1) val memoryManager = new StaticMemoryManager( conf, - maxExecutionMemory = Long.MaxValue, + maxOnHeapExecutionMemory = Long.MaxValue, maxStorageMemory = 1200, numCores = 1) store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, From f3f44fed63399992c7604a9db5b6d7810d57e286 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 28 Oct 2015 17:49:39 -0700 Subject: [PATCH 03/31] MemoryManager internals refactoring (WIP) --- .../scala/org/apache/spark/SparkEnv.scala | 2 +- ...trator.scala => ExecutionMemoryPool.scala} | 42 +++-- .../apache/spark/memory/MemoryManager.scala | 162 +++++------------- .../org/apache/spark/memory/MemoryPool.scala | 38 ++++ .../spark/memory/StaticMemoryManager.scala | 74 ++------ .../spark/memory/StorageMemoryPool.scala | 114 ++++++++++++ .../spark/memory/UnifiedMemoryManager.scala | 110 ++++-------- .../memory/GrantEverythingMemoryManager.scala | 27 ++- .../memory/StaticMemoryManagerSuite.scala | 23 +-- .../memory/UnifiedMemoryManagerSuite.scala | 35 ++-- 10 files changed, 302 insertions(+), 325 deletions(-) rename core/src/main/scala/org/apache/spark/memory/{CrossTaskMemoryArbitrator.scala => ExecutionMemoryPool.scala} (81%) create mode 100644 core/src/main/scala/org/apache/spark/memory/MemoryPool.scala create mode 100644 core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 398e0936906a3..e0c52c2ec81c1 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -338,7 +338,7 @@ object SparkEnv extends Logging { if (useLegacyMemoryManager) { new StaticMemoryManager(conf, numUsableCores) } else { - new UnifiedMemoryManager(conf, numUsableCores) + UnifiedMemoryManager(conf, numUsableCores) } val blockTransferService = new NettyBlockTransferService(conf, securityManager, numUsableCores) diff --git a/core/src/main/scala/org/apache/spark/memory/CrossTaskMemoryArbitrator.scala b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala similarity index 81% rename from core/src/main/scala/org/apache/spark/memory/CrossTaskMemoryArbitrator.scala rename to core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala index 416bc49a57fdb..ee39642632f33 100644 --- a/core/src/main/scala/org/apache/spark/memory/CrossTaskMemoryArbitrator.scala +++ b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala @@ -17,8 +17,6 @@ package org.apache.spark.memory -import javax.annotation.concurrent.{GuardedBy, ThreadSafe} - import scala.collection.mutable import org.apache.spark.{Logging, SparkException} @@ -30,28 +28,28 @@ import org.apache.spark.{Logging, SparkException} * to a large amount first and then causing others to spill to disk repeatedly. * * If there are N tasks, it ensures that each task 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 tasks and redo the calculations of 1 / 2N and 1 / N in waiting tasks whenever + * before it has to spill, and at most 1 / N. Because N varies dynamically,fting tasks whenever * this set changes. This is all done by synchronizing access to mutable state and using wait() and * notifyAll() to signal changes to callers. Prior to Spark 1.6, this arbitration of memory across * tasks was performed by the ShuffleMemoryManager. */ -@ThreadSafe -private[memory] class CrossTaskMemoryArbitrator( - memoryManager: MemoryManager, - maxMemory: () => Long, - poolName: String) extends Logging { +class ExecutionMemoryPool(poolName: String) extends MemoryPool with Logging { + + // TODO(josh): document and consider thread-safety contracts /** * Map from taskAttemptId -> memory consumption in bytes */ - @GuardedBy("memoryManager") private val memoryForTask = new mutable.HashMap[Long, Long]() + override def memoryUsed: Long = synchronized { + memoryForTask.values.sum + } + /** * Returns the memory consumption, in bytes, for the given task. */ - def getMemoryUsageForTask(taskAttemptId: Long): Long = memoryManager.synchronized { + def getMemoryUsageForTask(taskAttemptId: Long): Long = synchronized { memoryForTask.getOrElse(taskAttemptId, 0L) } @@ -66,7 +64,7 @@ private[memory] class CrossTaskMemoryArbitrator( * * @return the number of bytes granted to the task. */ - def acquireMemory(numBytes: Long, taskAttemptId: Long): Long = memoryManager.synchronized { + def acquireMemory(numBytes: Long, taskAttemptId: Long): Long = synchronized { assert(numBytes > 0, s"invalid number of bytes requested: $numBytes") // Add this task to the taskMemory map just so we can keep an accurate count of the number @@ -74,7 +72,7 @@ private[memory] class CrossTaskMemoryArbitrator( if (!memoryForTask.contains(taskAttemptId)) { memoryForTask(taskAttemptId) = 0L // This will later cause waiting tasks to wake up and check numTasks again - memoryManager.notifyAll() + notifyAll() } // Keep looping until we're either sure that we don't want to grant this request (because this @@ -84,26 +82,25 @@ private[memory] class CrossTaskMemoryArbitrator( while (true) { val numActiveTasks = memoryForTask.keys.size val curMem = memoryForTask(taskAttemptId) - val freeMemory = maxMemory() - memoryForTask.values.sum // How much we can grant this task; don't let it grow to more than 1 / numActiveTasks; // don't let it be negative val maxToGrant = - math.min(numBytes, math.max(0, (maxMemory() / numActiveTasks) - curMem)) + math.min(numBytes, math.max(0, (poolSize / numActiveTasks) - curMem)) // Only give it as much memory as is free, which might be none if it reached 1 / numTasks - val toGrant = math.min(maxToGrant, freeMemory) + val toGrant = math.min(maxToGrant, memoryFree) - if (curMem < maxMemory() / (2 * numActiveTasks)) { + if (curMem < poolSize / (2 * numActiveTasks)) { // We want to let each task get at least 1 / (2 * numActiveTasks) before blocking; // if we can't give it this much now, wait for other tasks to free up memory // (this happens if older tasks allocated lots of memory before N grew) - if (freeMemory >= math.min(maxToGrant, maxMemory() / (2 * numActiveTasks) - curMem)) { + if (memoryFree >= math.min(maxToGrant, poolSize / (2 * numActiveTasks) - curMem)) { memoryForTask(taskAttemptId) += toGrant return toGrant } else { logInfo( s"TID $taskAttemptId waiting for at least 1/2N of $poolName pool to be free") - memoryManager.wait() + wait() } } else { memoryForTask(taskAttemptId) += toGrant @@ -113,7 +110,7 @@ private[memory] class CrossTaskMemoryArbitrator( 0L // Never reached } - def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = memoryManager.synchronized { + def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized { val curMem = memoryForTask.getOrElse(taskAttemptId, 0L) if (curMem < numBytes) { throw new SparkException( @@ -126,16 +123,17 @@ private[memory] class CrossTaskMemoryArbitrator( memoryForTask.remove(taskAttemptId) } } - memoryManager.notifyAll() // Notify waiters in acquireMemory() that memory has been freed + notifyAll() // Notify waiters in acquireMemory() that memory has been freed } /** * Release all memory for the given task and mark it as inactive (e.g. when a task ends). * @return the number of bytes freed. */ - def releaseAllMemoryForTask(taskAttemptId: Long): Long = memoryManager.synchronized { + def releaseAllMemoryForTask(taskAttemptId: Long): Long = synchronized { val numBytesToFree = getMemoryUsageForTask(taskAttemptId) releaseMemory(numBytesToFree, taskAttemptId) numBytesToFree } + } diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 36419c7fafd8e..37d7eafedb2ba 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -17,14 +17,9 @@ package org.apache.spark.memory -import javax.annotation.concurrent.GuardedBy - import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - -import com.google.common.annotations.VisibleForTesting -import org.apache.spark.{Logging, SparkConf, TaskContext} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore} import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.memory.MemoryAllocator @@ -36,50 +31,35 @@ import org.apache.spark.unsafe.memory.MemoryAllocator * sorts and aggregations, while storage memory refers to that used for caching and propagating * internal data across the cluster. There exists one MemoryManager per JVM. */ -private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) extends Logging { +private[spark] abstract class MemoryManager( + conf: SparkConf, + numCores: Int, + maxOnHeapExecutionMemory: Long) extends Logging { // -- Methods related to memory allocation policies and bookkeeping ------------------------------ - // The memory store used to evict cached blocks - private var _memoryStore: MemoryStore = _ - protected def memoryStore: MemoryStore = { - if (_memoryStore == null) { - throw new IllegalArgumentException("memory store not initialized yet") - } - _memoryStore - } + // TODO(josh): think through and document thread-safety contracts + protected val storageMemoryPool = new StorageMemoryPool() + protected val onHeapExecutionMemoryPool = new ExecutionMemoryPool("on-heap execution") + protected val offHeapExecutionMemoryPool = new ExecutionMemoryPool("off-heap execution") - // Amount of execution/storage memory in use, accesses must be synchronized on `this` - @GuardedBy("this") protected var _onHeapExecutionMemoryUsed: Long = 0 - @GuardedBy("this") protected var _offHeapExecutionMemoryUsed: Long = 0 - @GuardedBy("this") protected var _storageMemoryUsed: Long = 0 - @GuardedBy("this") private[this] val onHeapExecutionMemoryArbitrator: CrossTaskMemoryArbitrator = - new CrossTaskMemoryArbitrator(this, maxOnHeapExecutionMemory _, "on-heap execution") - @GuardedBy("this") private[this] val offHeapExecutionMemoryArbitrator: CrossTaskMemoryArbitrator = - new CrossTaskMemoryArbitrator(this, maxOffHeapExecutionMemory _, "off-heap execution") + offHeapExecutionMemoryPool.incrementPoolSize(conf.getSizeAsBytes("spark.memory.offHeapSize", 0)) - /** - * Set the [[MemoryStore]] used by this manager to evict cached blocks. - * This must be set after construction due to initialization ordering constraints. - */ - final def setMemoryStore(store: MemoryStore): Unit = { - _memoryStore = store - } /** - * Total on-heap memory available for execution, in bytes. + * Total available memory for storage, in bytes. This amount can vary over time, depending on + * the MemoryManager implementation. + * In this model, this is equivalent to the amount of memory not occupied by execution. */ - def maxOnHeapExecutionMemory: Long - - /** - * Total off-heap memory available for execution, in bytes. - */ - final def maxOffHeapExecutionMemory: Long = conf.getSizeAsBytes("spark.memory.offHeapSize", 0) + def maxStorageMemory: Long /** - * Total available memory for storage, in bytes. + * Set the [[MemoryStore]] used by this manager to evict cached blocks. + * This must be set after construction due to initialization ordering constraints. */ - def maxStorageMemory: Long + final def setMemoryStore(store: MemoryStore): Unit = { + storageMemoryPool.setMemoryStore(store) + } // TODO: avoid passing evicted blocks around to simplify method signatures (SPARK-10985) @@ -91,7 +71,9 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte def acquireStorageMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = { + storageMemoryPool.acquireMemory(blockId, numBytes, evictedBlocks) + } /** * Acquire N bytes of memory to unroll the given block, evicting existing ones if necessary. @@ -106,19 +88,7 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte def acquireUnrollMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { - acquireStorageMemory(blockId, numBytes, evictedBlocks) - } - - /** - * Acquire N bytes of memory for execution, evicting cached blocks if necessary. - * Blocks evicted in the process, if any, are added to `evictedBlocks`. - * @return number of bytes successfully granted (<= N). - */ - @VisibleForTesting - private[memory] def doAcquireExecutionMemory( - numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean /** * Try to acquire up to `numBytes` of on-heap execution memory for the current task and return the @@ -128,29 +98,9 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte * task has a chance to ramp up to at least 1 / 2N of the total memory pool (where N is the # of * active tasks) before it is forced to spill. This can happen if the number of tasks increase * but an older task had a lot of memory already. - * - * Subclasses should override `doAcquireExecutionMemory` in order to customize the policies - * that control global sharing of memory between execution and storage. */ private[memory] - final def acquireOnHeapExecutionMemory(numBytes: Long, taskAttemptId: Long): Long = synchronized { - val memoryGranted = onHeapExecutionMemoryArbitrator.acquireMemory(numBytes, taskAttemptId) - if (memoryGranted > 0L) { - val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - val acquired = doAcquireExecutionMemory(memoryGranted, evictedBlocks) - assert(acquired <= memoryGranted) - onHeapExecutionMemoryArbitrator.releaseMemory(memoryGranted - acquired, taskAttemptId) - // Register evicted blocks, if any, with the active task metrics - Option(TaskContext.get()).foreach { tc => - val metrics = tc.taskMetrics() - val lastUpdatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) - metrics.updatedBlocks = Some(lastUpdatedBlocks ++ evictedBlocks.toSeq) - } - acquired - } else { - 0L - } - } + def acquireOnHeapExecutionMemory(numBytes: Long, taskAttemptId: Long): Long /** * Try to acquire up to `numBytes` of off-heap execution memory for the current task and return @@ -165,51 +115,25 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte def acquireOffHeapExecutionMemory( numBytes: Long, taskAttemptId: Long): Long = synchronized { - val memoryGranted = offHeapExecutionMemoryArbitrator.acquireMemory(numBytes, taskAttemptId) - _offHeapExecutionMemoryUsed += memoryGranted - memoryGranted - } - - @VisibleForTesting - private[memory] def releaseOnHeapExecutionMemory(numBytes: Long): Unit = synchronized { - if (numBytes > _onHeapExecutionMemoryUsed) { - logWarning(s"Attempted to release $numBytes bytes of on-heap execution " + - s"memory when we only have ${_onHeapExecutionMemoryUsed} bytes") - _onHeapExecutionMemoryUsed = 0 - } else { - _onHeapExecutionMemoryUsed -= numBytes - } - } - - @VisibleForTesting - private[memory] def releaseOffHeapExecutionMemory(numBytes: Long): Unit = synchronized { - if (numBytes > _offHeapExecutionMemoryUsed) { - logWarning(s"Attempted to release $numBytes bytes of off-heap execution " + - s"memory when we only have ${_offHeapExecutionMemoryUsed} bytes") - _offHeapExecutionMemoryUsed = 0 - } else { - _offHeapExecutionMemoryUsed -= numBytes - } + offHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) } /** * Release numBytes of on-heap execution memory belonging to the given task. */ private[memory] - final def releaseOnHeapExecutionMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized { - onHeapExecutionMemoryArbitrator.releaseMemory(numBytes, taskAttemptId) - releaseOnHeapExecutionMemory(numBytes) + def releaseOnHeapExecutionMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized { + onHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId) } /** * Release numBytes of off-heap execution memory belonging to the given task. */ private[memory] - final def releaseOffHeapExecutionMemory( + def releaseOffHeapExecutionMemory( numBytes: Long, taskAttemptId: Long): Unit = synchronized { - offHeapExecutionMemoryArbitrator.releaseMemory(numBytes, taskAttemptId) - releaseOffHeapExecutionMemory(numBytes) + offHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId) } /** @@ -217,37 +141,28 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte * @return the number of bytes freed. */ private[memory] def releaseAllExecutionMemoryForTask(taskAttemptId: Long): Long = synchronized { - val freedOnHeap = onHeapExecutionMemoryArbitrator.releaseAllMemoryForTask(taskAttemptId) - releaseOnHeapExecutionMemory(freedOnHeap) - val freedOffHeap = offHeapExecutionMemoryArbitrator.releaseAllMemoryForTask(taskAttemptId) - releaseOffHeapExecutionMemory(freedOffHeap) - freedOnHeap + freedOffHeap + onHeapExecutionMemoryPool.releaseAllMemoryForTask(taskAttemptId) + + offHeapExecutionMemoryPool.releaseAllMemoryForTask(taskAttemptId) } /** * Release N bytes of storage memory. */ def releaseStorageMemory(numBytes: Long): Unit = synchronized { - if (numBytes > _storageMemoryUsed) { - logWarning(s"Attempted to release $numBytes bytes of storage " + - s"memory when we only have ${_storageMemoryUsed} bytes") - _storageMemoryUsed = 0 - } else { - _storageMemoryUsed -= numBytes - } + storageMemoryPool.releaseMemory(numBytes) } /** * Release all storage memory acquired. */ - def releaseAllStorageMemory(): Unit = synchronized { - _storageMemoryUsed = 0 + final def releaseAllStorageMemory(): Unit = synchronized { + storageMemoryPool.releaseAllMemory() } /** * Release N bytes of unroll memory. */ - def releaseUnrollMemory(numBytes: Long): Unit = synchronized { + final def releaseUnrollMemory(numBytes: Long): Unit = synchronized { releaseStorageMemory(numBytes) } @@ -255,21 +170,22 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte * Execution memory currently in use, in bytes. */ final def executionMemoryUsed: Long = synchronized { - _onHeapExecutionMemoryUsed + onHeapExecutionMemoryPool.memoryUsed + offHeapExecutionMemoryPool.memoryUsed } /** * Storage memory currently in use, in bytes. */ final def storageMemoryUsed: Long = synchronized { - _storageMemoryUsed + storageMemoryPool.memoryUsed } /** * Returns the execution memory consumption, in bytes, for the given task. */ private[memory] def getExecutionMemoryUsageForTask(taskAttemptId: Long): Long = synchronized { - onHeapExecutionMemoryArbitrator.getMemoryUsageForTask(taskAttemptId) + onHeapExecutionMemoryPool.getMemoryUsageForTask(taskAttemptId) + + offHeapExecutionMemoryPool.getMemoryUsageForTask(taskAttemptId) } // -- Fields related to Tungsten managed memory ------------------------------------------------- diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala new file mode 100644 index 0000000000000..6822b181d4229 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala @@ -0,0 +1,38 @@ +/* + * 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.memory + +abstract class MemoryPool { + + private[this] var _poolSize: Long = 0 + + final def poolSize: Long = _poolSize + final def memoryFree: Long = _poolSize - memoryUsed + def memoryUsed: Long + + def incrementPoolSize(delta: Long): Unit = { + require(delta >= 0) + _poolSize += delta + } + + def decrementPoolSize(delta: Long): Unit = { + require(delta >= 0) + require(delta <= _poolSize) + _poolSize -= delta + } +} diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala index c1e22f8551d50..daf2ea3ebd655 100644 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala @@ -22,7 +22,6 @@ import scala.collection.mutable import org.apache.spark.SparkConf import org.apache.spark.storage.{BlockId, BlockStatus} - /** * A [[MemoryManager]] that statically partitions the heap space into disjoint regions. * @@ -32,10 +31,10 @@ import org.apache.spark.storage.{BlockId, BlockStatus} */ private[spark] class StaticMemoryManager( conf: SparkConf, - override val maxOnHeapExecutionMemory: Long, + maxOnHeapExecutionMemory: Long, override val maxStorageMemory: Long, numCores: Int) - extends MemoryManager(conf, numCores) { + extends MemoryManager(conf, numCores, maxOnHeapExecutionMemory) { def this(conf: SparkConf, numCores: Int) { this( @@ -45,81 +44,29 @@ private[spark] class StaticMemoryManager( numCores) } + onHeapExecutionMemoryPool.incrementPoolSize(maxOnHeapExecutionMemory) + storageMemoryPool.incrementPoolSize(maxStorageMemory) + // Max number of bytes worth of blocks to evict when unrolling private val maxMemoryToEvictForUnroll: Long = { (maxStorageMemory * conf.getDouble("spark.storage.unrollFraction", 0.2)).toLong } - /** - * Acquire N bytes of memory for execution. - * @return number of bytes successfully granted (<= N). - */ - override def doAcquireExecutionMemory( + override private[memory] def acquireOnHeapExecutionMemory( numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long = synchronized { - assert(numBytes >= 0) - assert(_onHeapExecutionMemoryUsed <= maxOnHeapExecutionMemory) - val bytesToGrant = math.min(numBytes, maxOnHeapExecutionMemory - _onHeapExecutionMemoryUsed) - _onHeapExecutionMemoryUsed += bytesToGrant - bytesToGrant - } - - /** - * Acquire N bytes of memory to cache the given block, evicting existing ones if necessary. - * Blocks evicted in the process, if any, are added to `evictedBlocks`. - * @return whether all N bytes were successfully granted. - */ - override def acquireStorageMemory( - blockId: BlockId, - numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { - acquireStorageMemory(blockId, numBytes, numBytes, evictedBlocks) + taskAttemptId: Long): Long = { + onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) } - /** - * Acquire N bytes of memory to unroll the given block, evicting existing ones if necessary. - * - * This evicts at most M bytes worth of existing blocks, where M is a fraction of the storage - * space specified by `spark.storage.unrollFraction`. Blocks evicted in the process, if any, - * are added to `evictedBlocks`. - * - * @return whether all N bytes were successfully granted. - */ override def acquireUnrollMemory( blockId: BlockId, numBytes: Long, evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { - val currentUnrollMemory = memoryStore.currentUnrollMemory + val currentUnrollMemory = storageMemoryPool.memoryStore.currentUnrollMemory val maxNumBytesToFree = math.max(0, maxMemoryToEvictForUnroll - currentUnrollMemory) val numBytesToFree = math.min(numBytes, maxNumBytesToFree) - acquireStorageMemory(blockId, numBytes, numBytesToFree, evictedBlocks) - } - - /** - * Acquire N bytes of storage memory for the given block, evicting existing ones if necessary. - * - * @param blockId the ID of the block we are acquiring storage memory for - * @param numBytesToAcquire the size of this block - * @param numBytesToFree the size of space to be freed through evicting blocks - * @param evictedBlocks a holder for blocks evicted in the process - * @return whether all N bytes were successfully granted. - */ - private def acquireStorageMemory( - blockId: BlockId, - numBytesToAcquire: Long, - numBytesToFree: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { - assert(numBytesToAcquire >= 0) - assert(numBytesToFree >= 0) - memoryStore.ensureFreeSpace(blockId, numBytesToFree, evictedBlocks) - assert(_storageMemoryUsed <= maxStorageMemory) - val enoughMemory = _storageMemoryUsed + numBytesToAcquire <= maxStorageMemory - if (enoughMemory) { - _storageMemoryUsed += numBytesToAcquire - } - enoughMemory + storageMemoryPool.acquireMemory(blockId, numBytes, numBytesToFree, evictedBlocks) } - } @@ -135,7 +82,6 @@ private[spark] object StaticMemoryManager { (systemMaxMemory * memoryFraction * safetyFraction).toLong } - /** * Return the total amount of memory available for the execution region, in bytes. */ diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala new file mode 100644 index 0000000000000..550463295ac07 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -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.memory + +import org.apache.spark.{TaskContext, Logging} +import org.apache.spark.storage.{MemoryStore, BlockStatus, BlockId} + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +class StorageMemoryPool extends MemoryPool with Logging { + + private[this] var _memoryUsed: Long = 0L + + override def memoryUsed: Long = _memoryUsed + + private var _memoryStore: MemoryStore = _ + def memoryStore: MemoryStore = { + if (_memoryStore == null) { + throw new IllegalArgumentException("memory store not initialized yet") + } + _memoryStore + } + + /** + * Set the [[MemoryStore]] used by this manager to evict cached blocks. + * This must be set after construction due to initialization ordering constraints. + */ + final def setMemoryStore(store: MemoryStore): Unit = { + _memoryStore = store + } + + /** + * Acquire N bytes of memory to cache the given block, evicting existing ones if necessary. + * Blocks evicted in the process, if any, are added to `evictedBlocks`. + * @return whether all N bytes were successfully granted. + */ + def acquireMemory( + blockId: BlockId, + numBytes: Long, + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { + acquireMemory(blockId, numBytes, numBytes, evictedBlocks) + } + + /** + * Acquire N bytes of storage memory for the given block, evicting existing ones if necessary. + * + * @param blockId the ID of the block we are acquiring storage memory for + * @param numBytesToAcquire the size of this block + * @param numBytesToFree the size of space to be freed through evicting blocks + * @return whether all N bytes were successfully granted. + */ + def acquireMemory( + blockId: BlockId, + numBytesToAcquire: Long, + numBytesToFree: Long, + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { + assert(numBytesToAcquire >= 0) + assert(numBytesToFree >= 0) + // TODO(josh): check whether there is enough memory / handle eviction + memoryStore.ensureFreeSpace(blockId, numBytesToFree, evictedBlocks) + // Register evicted blocks, if any, with the active task metrics + Option(TaskContext.get()).foreach { tc => + val metrics = tc.taskMetrics() + val lastUpdatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) + metrics.updatedBlocks = Some(lastUpdatedBlocks ++ evictedBlocks.toSeq) + } + // TODO(josh): is this assertion in the right place? + assert(memoryUsed <= poolSize) + val enoughMemory = numBytesToAcquire <= memoryFree + if (enoughMemory) { + _memoryUsed += numBytesToAcquire + } + enoughMemory + } + + def releaseMemory(size: Long): Unit = { + if (size > _memoryUsed) { + logWarning(s"Attempted to release $size bytes of storage " + + s"memory when we only have ${_memoryUsed} bytes") + _memoryUsed = 0 + } else { + _memoryUsed -= size + } + } + + def releaseAllMemory(): Unit = { + _memoryUsed = 0 + } + + def shrinkPoolByEvictingBlocks(spaceToEnsure: Long): Long = { + val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + memoryStore.ensureFreeSpace(spaceToEnsure, evictedBlocks) + val spaceFreed = evictedBlocks.map(_._2.memSize).sum + _memoryUsed -= spaceFreed + decrementPoolSize(spaceFreed) + spaceFreed + } +} diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 15b7e53ac2519..a0bbd2800a225 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -22,7 +22,6 @@ import scala.collection.mutable import org.apache.spark.SparkConf import org.apache.spark.storage.{BlockStatus, BlockId} - /** * A [[MemoryManager]] that enforces a soft boundary between execution and storage such that * either side can borrow memory from the other. @@ -41,98 +40,63 @@ import org.apache.spark.storage.{BlockStatus, BlockId} * The implication is that attempts to cache blocks may fail if execution has already eaten * up most of the storage space, in which case the new blocks will be evicted immediately * according to their respective storage levels. + * + * @param minimumStoragePoolSize Size of the storage region, in bytes. + * This region is not statically reserved; execution can borrow from + * it if necessary. Cached blocks can be evicted only if actual + * storage memory usage exceeds this region. */ -private[spark] class UnifiedMemoryManager( +private[spark] class UnifiedMemoryManager private[memory] ( conf: SparkConf, maxMemory: Long, + private val minimumStoragePoolSize: Long, numCores: Int) - extends MemoryManager(conf, numCores) { - - def this(conf: SparkConf, numCores: Int) { - this(conf, UnifiedMemoryManager.getMaxMemory(conf), numCores) - } + extends MemoryManager( + conf, + numCores, + maxOnHeapExecutionMemory = maxMemory - minimumStoragePoolSize) { - /** - * Size of the storage region, in bytes. - * - * This region is not statically reserved; execution can borrow from it if necessary. - * Cached blocks can be evicted only if actual storage memory usage exceeds this region. - */ - private val storageRegionSize: Long = { - (maxMemory * conf.getDouble("spark.memory.storageFraction", 0.5)).toLong - } - - /** - * Total amount of memory, in bytes, not currently occupied by either execution or storage. - */ - private def totalFreeMemory: Long = synchronized { - assert(_onHeapExecutionMemoryUsed <= maxMemory) - assert(_storageMemoryUsed <= maxMemory) - assert(_onHeapExecutionMemoryUsed + _storageMemoryUsed <= maxMemory) - maxMemory - _onHeapExecutionMemoryUsed - _storageMemoryUsed - } - - /** - * Total available memory for execution, in bytes. - * In this model, this is equivalent to the amount of memory not occupied by storage. - */ - override def maxOnHeapExecutionMemory: Long = synchronized { - maxMemory - _storageMemoryUsed - } + onHeapExecutionMemoryPool.incrementPoolSize(maxMemory - minimumStoragePoolSize) + storageMemoryPool.incrementPoolSize(minimumStoragePoolSize) - /** - * Total available memory for storage, in bytes. - * In this model, this is equivalent to the amount of memory not occupied by execution. - */ - override def maxStorageMemory: Long = synchronized { - maxMemory - _onHeapExecutionMemoryUsed + override def maxStorageMemory: Long = { + maxMemory - onHeapExecutionMemoryPool.memoryUsed } - /** - * Acquire N bytes of memory for execution, evicting cached blocks if necessary. - * - * This method evicts blocks only up to the amount of memory borrowed by storage. - * Blocks evicted in the process, if any, are added to `evictedBlocks`. - * @return number of bytes successfully granted (<= N). - */ - private[memory] override def doAcquireExecutionMemory( + private[memory] def acquireOnHeapExecutionMemory( numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long = synchronized { + taskAttemptId: Long): Long = synchronized { assert(numBytes >= 0) - val memoryBorrowedByStorage = math.max(0, _storageMemoryUsed - storageRegionSize) + val memoryBorrowedByStorage = math.max(0, storageMemoryPool.memoryUsed - minimumStoragePoolSize) // If there is not enough free memory AND storage has borrowed some execution memory, // then evict as much memory borrowed by storage as needed to grant this request - val shouldEvictStorage = totalFreeMemory < numBytes && memoryBorrowedByStorage > 0 - if (shouldEvictStorage) { - val spaceToEnsure = math.min(numBytes, memoryBorrowedByStorage) - memoryStore.ensureFreeSpace(spaceToEnsure, evictedBlocks) + if (numBytes > onHeapExecutionMemoryPool.memoryFree && memoryBorrowedByStorage > 0) { + val spaceReclaimed = + storageMemoryPool.shrinkPoolByEvictingBlocks(math.min(numBytes, memoryBorrowedByStorage)) + onHeapExecutionMemoryPool.incrementPoolSize(spaceReclaimed) } - val bytesToGrant = math.min(numBytes, totalFreeMemory) - _onHeapExecutionMemoryUsed += bytesToGrant - bytesToGrant + onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) } - /** - * Acquire N bytes of memory to cache the given block, evicting existing ones if necessary. - * Blocks evicted in the process, if any, are added to `evictedBlocks`. - * @return whether all N bytes were successfully granted. - */ - override def acquireStorageMemory( + override def acquireUnrollMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { - assert(numBytes >= 0) - memoryStore.ensureFreeSpace(blockId, numBytes, evictedBlocks) - val enoughMemory = totalFreeMemory >= numBytes - if (enoughMemory) { - _storageMemoryUsed += numBytes - } - enoughMemory + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = { + acquireStorageMemory(blockId, numBytes, evictedBlocks) } - } -private object UnifiedMemoryManager { +object UnifiedMemoryManager { + + def apply(conf: SparkConf, numCores: Int): UnifiedMemoryManager = { + val maxMemory = getMaxMemory(conf) + new UnifiedMemoryManager( + conf, + maxMemory = maxMemory, + minimumStoragePoolSize = + (maxMemory * conf.getDouble("spark.memory.storageFraction", 0.5)).toLong, + numCores = numCores) + } /** * Return the total amount of memory shared between execution and storage, in bytes. diff --git a/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala index b9203b2604d67..7817ef92edb18 100644 --- a/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala +++ b/core/src/test/scala/org/apache/spark/memory/GrantEverythingMemoryManager.scala @@ -22,27 +22,22 @@ import scala.collection.mutable import org.apache.spark.SparkConf import org.apache.spark.storage.{BlockStatus, BlockId} -class GrantEverythingMemoryManager(conf: SparkConf) extends MemoryManager(conf, numCores = 1) { - private[memory] override def doAcquireExecutionMemory( - numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long = synchronized { +class GrantEverythingMemoryManager(conf: SparkConf) + extends MemoryManager(conf, numCores = 1, Long.MaxValue) { + override private[memory] def acquireOnHeapExecutionMemory( + numBytes: Long, + taskAttemptId: Long): Long = { if (oom) { oom = false 0 } else { - _onHeapExecutionMemoryUsed += numBytes // To suppress warnings when freeing unallocated memory numBytes } } - // TODO(josh): this code is becoming very confusing; refactoring needed - override def acquireOffHeapExecutionMemory(numBytes: Long, taskId: Long): Long = synchronized { - if (oom) { - oom = false - 0 - } else { - _offHeapExecutionMemoryUsed += numBytes // To suppress warnings when freeing unallocated memory - numBytes - } + override private[memory] def acquireOffHeapExecutionMemory( + numBytes: Long, + taskAttemptId: Long): Long = { + acquireOnHeapExecutionMemory(numBytes, taskAttemptId) } override def acquireStorageMemory( blockId: BlockId, @@ -53,11 +48,11 @@ class GrantEverythingMemoryManager(conf: SparkConf) extends MemoryManager(conf, numBytes: Long, evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = true override def releaseStorageMemory(numBytes: Long): Unit = { } - override def maxOnHeapExecutionMemory: Long = Long.MaxValue + override private[memory] def releaseOnHeapExecutionMemory(numBytes: Long, tid: Long): Unit = {} + override private[memory] def releaseOffHeapExecutionMemory(numBytes: Long, tid: Long): Unit = {} override def maxStorageMemory: Long = Long.MaxValue private var oom = false - def markExecutionAsOutOfMemory(): Unit = { oom = true } diff --git a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala index 4748edbc5baf4..3ba73e25d1842 100644 --- a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala @@ -24,7 +24,6 @@ import org.mockito.Mockito.when import org.apache.spark.SparkConf import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore, TestBlockId} - class StaticMemoryManagerSuite extends MemoryManagerSuite { private val conf = new SparkConf().set("spark.storage.unrollFraction", "0.4") private val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -51,23 +50,24 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { test("basic execution memory") { val maxExecutionMem = 1000L + val taskAttemptId = 0L val (mm, _) = makeThings(maxExecutionMem, Long.MaxValue) assert(mm.executionMemoryUsed === 0L) - assert(mm.doAcquireExecutionMemory(10L, evictedBlocks) === 10L) + assert(mm.acquireOnHeapExecutionMemory(10L, taskAttemptId) === 10L) assert(mm.executionMemoryUsed === 10L) - assert(mm.doAcquireExecutionMemory(100L, evictedBlocks) === 100L) + assert(mm.acquireOnHeapExecutionMemory(100L, taskAttemptId) === 100L) // Acquire up to the max - assert(mm.doAcquireExecutionMemory(1000L, evictedBlocks) === 890L) + assert(mm.acquireOnHeapExecutionMemory(1000L, taskAttemptId) === 890L) assert(mm.executionMemoryUsed === maxExecutionMem) - assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 0L) + assert(mm.acquireOnHeapExecutionMemory(1L, taskAttemptId) === 0L) assert(mm.executionMemoryUsed === maxExecutionMem) - mm.releaseOnHeapExecutionMemory(800L) + mm.releaseOnHeapExecutionMemory(800L, taskAttemptId) assert(mm.executionMemoryUsed === 200L) // Acquire after release - assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 1L) + assert(mm.acquireOnHeapExecutionMemory(1L, taskAttemptId) === 1L) assert(mm.executionMemoryUsed === 201L) // Release beyond what was acquired - mm.releaseOnHeapExecutionMemory(maxExecutionMem) + mm.releaseOnHeapExecutionMemory(maxExecutionMem, taskAttemptId) assert(mm.executionMemoryUsed === 0L) } @@ -113,13 +113,14 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { test("execution and storage isolation") { val maxExecutionMem = 200L val maxStorageMem = 1000L + val taskAttemptId = 0L val dummyBlock = TestBlockId("ain't nobody love like you do") val (mm, ms) = makeThings(maxExecutionMem, maxStorageMem) // Only execution memory should increase - assert(mm.doAcquireExecutionMemory(100L, evictedBlocks) === 100L) + assert(mm.acquireOnHeapExecutionMemory(100L, taskAttemptId) === 100L) assert(mm.storageMemoryUsed === 0L) assert(mm.executionMemoryUsed === 100L) - assert(mm.doAcquireExecutionMemory(1000L, evictedBlocks) === 100L) + assert(mm.acquireOnHeapExecutionMemory(1000L, taskAttemptId) === 100L) assert(mm.storageMemoryUsed === 0L) assert(mm.executionMemoryUsed === 200L) // Only storage memory should increase @@ -128,7 +129,7 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { assert(mm.storageMemoryUsed === 50L) assert(mm.executionMemoryUsed === 200L) // Only execution memory should be released - mm.releaseOnHeapExecutionMemory(133L) + mm.releaseOnHeapExecutionMemory(133L, taskAttemptId) assert(mm.storageMemoryUsed === 50L) assert(mm.executionMemoryUsed === 67L) // Only storage memory should be released diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 1c45857eda442..f0b3603a3a4fb 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -34,13 +34,15 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes * Make a [[UnifiedMemoryManager]] and a [[MemoryStore]] with limited class dependencies. */ private def makeThings(maxMemory: Long): (UnifiedMemoryManager, MemoryStore) = { - val mm = new UnifiedMemoryManager(conf, maxMemory, numCores = 1) + val mm = new + UnifiedMemoryManager(conf, maxMemory, minimumStoragePoolSize = maxMemory /2, numCores = 1) val ms = makeMemoryStore(mm) (mm, ms) } override protected def createMemoryManager(maxMemory: Long): MemoryManager = { - new UnifiedMemoryManager(conf, maxMemory, numCores = 1) + new UnifiedMemoryManager( + conf, maxMemory, minimumStoragePoolSize = maxMemory /2 , numCores = 1) } private def getStorageRegionSize(mm: UnifiedMemoryManager): Long = { @@ -58,23 +60,24 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes test("basic execution memory") { val maxMemory = 1000L + val taskAttemptId = 0L val (mm, _) = makeThings(maxMemory) assert(mm.executionMemoryUsed === 0L) - assert(mm.doAcquireExecutionMemory(10L, evictedBlocks) === 10L) + assert(mm.acquireOnHeapExecutionMemory(10L, taskAttemptId) === 10L) assert(mm.executionMemoryUsed === 10L) - assert(mm.doAcquireExecutionMemory(100L, evictedBlocks) === 100L) + assert(mm.acquireOnHeapExecutionMemory(100L, taskAttemptId) === 100L) // Acquire up to the max - assert(mm.doAcquireExecutionMemory(1000L, evictedBlocks) === 890L) + assert(mm.acquireOnHeapExecutionMemory(1000L, taskAttemptId) === 890L) assert(mm.executionMemoryUsed === maxMemory) - assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 0L) + assert(mm.acquireOnHeapExecutionMemory(1L, taskAttemptId) === 0L) assert(mm.executionMemoryUsed === maxMemory) - mm.releaseOnHeapExecutionMemory(800L) + mm.releaseOnHeapExecutionMemory(800L, taskAttemptId) assert(mm.executionMemoryUsed === 200L) // Acquire after release - assert(mm.doAcquireExecutionMemory(1L, evictedBlocks) === 1L) + assert(mm.acquireOnHeapExecutionMemory(1L, taskAttemptId) === 1L) assert(mm.executionMemoryUsed === 201L) // Release beyond what was acquired - mm.releaseOnHeapExecutionMemory(maxMemory) + mm.releaseOnHeapExecutionMemory(maxMemory, taskAttemptId) assert(mm.executionMemoryUsed === 0L) } @@ -118,6 +121,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes test("execution evicts storage") { val maxMemory = 1000L + val taskAttemptId = 0L val (mm, ms) = makeThings(maxMemory) // First, ensure the test classes are set up as expected val expectedStorageRegionSize = 500L @@ -136,12 +140,12 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes require(mm.storageMemoryUsed > storageRegionSize, s"bad test: storage memory used should exceed the storage region") // Execution needs to request 250 bytes to evict storage memory - assert(mm.doAcquireExecutionMemory(100L, evictedBlocks) === 100L) + assert(mm.acquireOnHeapExecutionMemory(100L, taskAttemptId) === 100L) assert(mm.executionMemoryUsed === 100L) assert(mm.storageMemoryUsed === 750L) assertEnsureFreeSpaceNotCalled(ms) // Execution wants 200 bytes but only 150 are free, so storage is evicted - assert(mm.doAcquireExecutionMemory(200L, evictedBlocks) === 200L) + assert(mm.acquireOnHeapExecutionMemory(200L, taskAttemptId) === 200L) assertEnsureFreeSpaceCalled(ms, 200L) assert(mm.executionMemoryUsed === 300L) mm.releaseAllStorageMemory() @@ -155,7 +159,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes s"bad test: storage memory used should be within the storage region") // Execution cannot evict storage because the latter is within the storage fraction, // so grant only what's remaining without evicting anything, i.e. 1000 - 300 - 400 = 300 - assert(mm.doAcquireExecutionMemory(400L, evictedBlocks) === 300L) + assert(mm.acquireOnHeapExecutionMemory(400L, taskAttemptId) === 300L) assert(mm.executionMemoryUsed === 600L) assert(mm.storageMemoryUsed === 400L) assertEnsureFreeSpaceNotCalled(ms) @@ -163,6 +167,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes test("storage does not evict execution") { val maxMemory = 1000L + val taskAttemptId = 0L val (mm, ms) = makeThings(maxMemory) // First, ensure the test classes are set up as expected val expectedStorageRegionSize = 500L @@ -174,7 +179,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes require(executionRegionSize === expectedExecutionRegionSize, "bad test: storage region size is unexpected") // Acquire enough execution memory to exceed the execution region - assert(mm.doAcquireExecutionMemory(800L, evictedBlocks) === 800L) + assert(mm.acquireOnHeapExecutionMemory(800L, taskAttemptId) === 800L) assert(mm.executionMemoryUsed === 800L) assert(mm.storageMemoryUsed === 0L) assertEnsureFreeSpaceNotCalled(ms) @@ -189,10 +194,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes assert(mm.executionMemoryUsed === 800L) assert(mm.storageMemoryUsed === 100L) assertEnsureFreeSpaceCalled(ms, 250L) - mm.releaseOnHeapExecutionMemory(maxMemory) + mm.releaseOnHeapExecutionMemory(maxMemory, taskAttemptId) mm.releaseStorageMemory(maxMemory) // Acquire some execution memory again, but this time keep it within the execution region - assert(mm.doAcquireExecutionMemory(200L, evictedBlocks) === 200L) + assert(mm.acquireOnHeapExecutionMemory(200L, taskAttemptId) === 200L) assert(mm.executionMemoryUsed === 200L) assert(mm.storageMemoryUsed === 0L) assertEnsureFreeSpaceNotCalled(ms) From eb3180aa94be6e5cffa637a5e97ab2cd6f266569 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 28 Oct 2015 22:57:02 -0700 Subject: [PATCH 04/31] Gradually fix test issues; address thread-safety (WIP). --- .../spark/memory/ExecutionMemoryPool.scala | 42 ++++++++------ .../apache/spark/memory/MemoryManager.scala | 5 +- .../spark/memory/StaticMemoryManager.scala | 2 +- .../spark/memory/StorageMemoryPool.scala | 6 +- .../spark/memory/UnifiedMemoryManager.scala | 19 +++++-- .../spark/memory/MemoryManagerSuite.scala | 1 + .../memory/UnifiedMemoryManagerSuite.scala | 56 ++++++++++--------- 7 files changed, 76 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala index ee39642632f33..044d472a628b8 100644 --- a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala @@ -17,9 +17,11 @@ package org.apache.spark.memory +import javax.annotation.concurrent.GuardedBy + import scala.collection.mutable -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.Logging /** * Implements policies and bookkeeping for sharing a fixed-size pool of memory between tasks. @@ -28,28 +30,31 @@ import org.apache.spark.{Logging, SparkException} * to a large amount first and then causing others to spill to disk repeatedly. * * If there are N tasks, it ensures that each task can acquire at least 1 / 2N of the memory - * before it has to spill, and at most 1 / N. Because N varies dynamically,fting tasks whenever - * this set changes. This is all done by synchronizing access to mutable state and using wait() and + * before it has to spill, and at most 1 / N. Because N varies dynamically, we keep track of the + * set of active tasks and redo the calculations of 1 / 2N and 1 / N in waiting tasks whenever this + * set changes. This is all done by synchronizing access to mutable state and using wait() and * notifyAll() to signal changes to callers. Prior to Spark 1.6, this arbitration of memory across * tasks was performed by the ShuffleMemoryManager. + * + * @param memoryManager a [[MemoryManager]] instance to synchronize on + * @param poolName a human-readable name for this pool, for use in log messages */ -class ExecutionMemoryPool(poolName: String) extends MemoryPool with Logging { - - // TODO(josh): document and consider thread-safety contracts +class ExecutionMemoryPool(memoryManager: Object, poolName: String) extends MemoryPool with Logging { /** * Map from taskAttemptId -> memory consumption in bytes */ + @GuardedBy("memoryManager") private val memoryForTask = new mutable.HashMap[Long, Long]() - override def memoryUsed: Long = synchronized { + override def memoryUsed: Long = memoryManager.synchronized { memoryForTask.values.sum } /** * Returns the memory consumption, in bytes, for the given task. */ - def getMemoryUsageForTask(taskAttemptId: Long): Long = synchronized { + def getMemoryUsageForTask(taskAttemptId: Long): Long = memoryManager.synchronized { memoryForTask.getOrElse(taskAttemptId, 0L) } @@ -64,7 +69,7 @@ class ExecutionMemoryPool(poolName: String) extends MemoryPool with Logging { * * @return the number of bytes granted to the task. */ - def acquireMemory(numBytes: Long, taskAttemptId: Long): Long = synchronized { + def acquireMemory(numBytes: Long, taskAttemptId: Long): Long = memoryManager.synchronized { assert(numBytes > 0, s"invalid number of bytes requested: $numBytes") // Add this task to the taskMemory map just so we can keep an accurate count of the number @@ -72,7 +77,7 @@ class ExecutionMemoryPool(poolName: String) extends MemoryPool with Logging { if (!memoryForTask.contains(taskAttemptId)) { memoryForTask(taskAttemptId) = 0L // This will later cause waiting tasks to wake up and check numTasks again - notifyAll() + memoryManager.notifyAll() } // Keep looping until we're either sure that we don't want to grant this request (because this @@ -100,7 +105,7 @@ class ExecutionMemoryPool(poolName: String) extends MemoryPool with Logging { } else { logInfo( s"TID $taskAttemptId waiting for at least 1/2N of $poolName pool to be free") - wait() + memoryManager.wait() } } else { memoryForTask(taskAttemptId) += toGrant @@ -110,27 +115,30 @@ class ExecutionMemoryPool(poolName: String) extends MemoryPool with Logging { 0L // Never reached } - def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized { + def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = memoryManager.synchronized { val curMem = memoryForTask.getOrElse(taskAttemptId, 0L) - if (curMem < numBytes) { - throw new SparkException( + var memoryToFree = if (curMem < numBytes) { + logWarning( s"Internal error: release called on $numBytes bytes but task only has $curMem bytes " + s"of memory from the $poolName pool") + curMem + } else { + numBytes } if (memoryForTask.contains(taskAttemptId)) { - memoryForTask(taskAttemptId) -= numBytes + memoryForTask(taskAttemptId) -= memoryToFree if (memoryForTask(taskAttemptId) <= 0) { memoryForTask.remove(taskAttemptId) } } - notifyAll() // Notify waiters in acquireMemory() that memory has been freed + memoryManager.notifyAll() // Notify waiters in acquireMemory() that memory has been freed } /** * Release all memory for the given task and mark it as inactive (e.g. when a task ends). * @return the number of bytes freed. */ - def releaseAllMemoryForTask(taskAttemptId: Long): Long = synchronized { + def releaseAllMemoryForTask(taskAttemptId: Long): Long = memoryManager.synchronized { val numBytesToFree = getMemoryUsageForTask(taskAttemptId) releaseMemory(numBytesToFree, taskAttemptId) numBytesToFree diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 37d7eafedb2ba..29dc54a4c4a43 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -40,12 +40,11 @@ private[spark] abstract class MemoryManager( // TODO(josh): think through and document thread-safety contracts protected val storageMemoryPool = new StorageMemoryPool() - protected val onHeapExecutionMemoryPool = new ExecutionMemoryPool("on-heap execution") - protected val offHeapExecutionMemoryPool = new ExecutionMemoryPool("off-heap execution") + protected val onHeapExecutionMemoryPool = new ExecutionMemoryPool(this, "on-heap execution") + protected val offHeapExecutionMemoryPool = new ExecutionMemoryPool(this, "off-heap execution") offHeapExecutionMemoryPool.incrementPoolSize(conf.getSizeAsBytes("spark.memory.offHeapSize", 0)) - /** * Total available memory for storage, in bytes. This amount can vary over time, depending on * the MemoryManager implementation. diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala index daf2ea3ebd655..054c676a1c8c5 100644 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala @@ -54,7 +54,7 @@ private[spark] class StaticMemoryManager( override private[memory] def acquireOnHeapExecutionMemory( numBytes: Long, - taskAttemptId: Long): Long = { + taskAttemptId: Long): Long = synchronized { onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) } diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index 550463295ac07..221a8e30e8616 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -89,7 +89,7 @@ class StorageMemoryPool extends MemoryPool with Logging { enoughMemory } - def releaseMemory(size: Long): Unit = { + def releaseMemory(size: Long): Unit = synchronized { if (size > _memoryUsed) { logWarning(s"Attempted to release $size bytes of storage " + s"memory when we only have ${_memoryUsed} bytes") @@ -99,11 +99,11 @@ class StorageMemoryPool extends MemoryPool with Logging { } } - def releaseAllMemory(): Unit = { + def releaseAllMemory(): Unit = synchronized { _memoryUsed = 0 } - def shrinkPoolByEvictingBlocks(spaceToEnsure: Long): Long = { + def shrinkPoolByEvictingBlocks(spaceToEnsure: Long): Long = synchronized { val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] memoryStore.ensureFreeSpace(spaceToEnsure, evictedBlocks) val spaceFreed = evictedBlocks.map(_._2.memSize).sum diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index a0bbd2800a225..ca6bb9a67ba0a 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -56,10 +56,9 @@ private[spark] class UnifiedMemoryManager private[memory] ( numCores, maxOnHeapExecutionMemory = maxMemory - minimumStoragePoolSize) { - onHeapExecutionMemoryPool.incrementPoolSize(maxMemory - minimumStoragePoolSize) - storageMemoryPool.incrementPoolSize(minimumStoragePoolSize) + onHeapExecutionMemoryPool.incrementPoolSize(maxMemory) - override def maxStorageMemory: Long = { + override def maxStorageMemory: Long = synchronized { maxMemory - onHeapExecutionMemoryPool.memoryUsed } @@ -78,10 +77,22 @@ private[spark] class UnifiedMemoryManager private[memory] ( onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) } + override def acquireStorageMemory( + blockId: BlockId, + numBytes: Long, + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { + if (numBytes > storageMemoryPool.memoryFree + && numBytes <= onHeapExecutionMemoryPool.memoryFree) { + onHeapExecutionMemoryPool.decrementPoolSize(numBytes) + storageMemoryPool.incrementPoolSize(numBytes) + } + storageMemoryPool.acquireMemory(blockId, numBytes, evictedBlocks) + } + override def acquireUnrollMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = { + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { acquireStorageMemory(blockId, numBytes, evictedBlocks) } } diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index a1cc3b8595fc9..a0f0eab70365f 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -146,6 +146,7 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val taskMemoryManager = new TaskMemoryManager(manager, 0) assert(taskMemoryManager.acquireOnHeapExecutionMemory(100L) === 100L) + assert(taskMemoryManager.getMemoryConsumptionForThisTask === 100L) assert(taskMemoryManager.acquireOnHeapExecutionMemory(400L) === 400L) assert(taskMemoryManager.acquireOnHeapExecutionMemory(400L) === 400L) assert(taskMemoryManager.acquireOnHeapExecutionMemory(200L) === 100L) diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index f0b3603a3a4fb..1fbd85b51caed 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -26,37 +26,39 @@ import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore, TestBlockId} class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTester { - private val conf = new SparkConf().set("spark.memory.storageFraction", "0.5") private val dummyBlock = TestBlockId("--") private val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + private val storageFraction: Double = 0.5 + /** * Make a [[UnifiedMemoryManager]] and a [[MemoryStore]] with limited class dependencies. */ private def makeThings(maxMemory: Long): (UnifiedMemoryManager, MemoryStore) = { - val mm = new - UnifiedMemoryManager(conf, maxMemory, minimumStoragePoolSize = maxMemory /2, numCores = 1) + val mm = createMemoryManager(maxMemory) val ms = makeMemoryStore(mm) (mm, ms) } - override protected def createMemoryManager(maxMemory: Long): MemoryManager = { - new UnifiedMemoryManager( - conf, maxMemory, minimumStoragePoolSize = maxMemory /2 , numCores = 1) + override protected def createMemoryManager(maxMemory: Long): UnifiedMemoryManager = { + val conf = new SparkConf() + .set("spark.memory.fraction", "1") + .set("spark.testing.memory", maxMemory.toString) + .set("spark.memory.storageFraction", storageFraction.toString) + UnifiedMemoryManager(conf, numCores = 1) } - private def getStorageRegionSize(mm: UnifiedMemoryManager): Long = { - mm invokePrivate PrivateMethod[Long]('storageRegionSize)() - } +// private def getStorageRegionSize(mm: UnifiedMemoryManager): Long = { +// (mm invokePrivate PrivateMethod[StorageMemoryPool]('storageMemoryPool)()).poolSize +// } - test("storage region size") { - val maxMemory = 1000L - val (mm, _) = makeThings(maxMemory) - val storageFraction = conf.get("spark.memory.storageFraction").toDouble - val expectedStorageRegionSize = maxMemory * storageFraction - val actualStorageRegionSize = getStorageRegionSize(mm) - assert(expectedStorageRegionSize === actualStorageRegionSize) - } +// test("storage region size") { +// val maxMemory = 1000L +// val (mm, _) = makeThings(maxMemory) +// val expectedStorageRegionSize = maxMemory * storageFraction +// val actualStorageRegionSize = getStorageRegionSize(mm) +// assert(expectedStorageRegionSize === actualStorageRegionSize) +// } test("basic execution memory") { val maxMemory = 1000L @@ -126,10 +128,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes // First, ensure the test classes are set up as expected val expectedStorageRegionSize = 500L val expectedExecutionRegionSize = 500L - val storageRegionSize = getStorageRegionSize(mm) +// val storageRegionSize = getStorageRegionSize(mm) val executionRegionSize = maxMemory - expectedStorageRegionSize - require(storageRegionSize === expectedStorageRegionSize, - "bad test: storage region size is unexpected") +// require(storageRegionSize === expectedStorageRegionSize, +// "bad test: storage region size is unexpected") require(executionRegionSize === expectedExecutionRegionSize, "bad test: storage region size is unexpected") // Acquire enough storage memory to exceed the storage region @@ -137,8 +139,8 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes assertEnsureFreeSpaceCalled(ms, 750L) assert(mm.executionMemoryUsed === 0L) assert(mm.storageMemoryUsed === 750L) - require(mm.storageMemoryUsed > storageRegionSize, - s"bad test: storage memory used should exceed the storage region") +// require(mm.storageMemoryUsed > storageRegionSize, +// s"bad test: storage memory used should exceed the storage region") // Execution needs to request 250 bytes to evict storage memory assert(mm.acquireOnHeapExecutionMemory(100L, taskAttemptId) === 100L) assert(mm.executionMemoryUsed === 100L) @@ -155,8 +157,8 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes // Acquire some storage memory again, but this time keep it within the storage region assert(mm.acquireStorageMemory(dummyBlock, 400L, evictedBlocks)) assertEnsureFreeSpaceCalled(ms, 400L) - require(mm.storageMemoryUsed < storageRegionSize, - s"bad test: storage memory used should be within the storage region") +// require(mm.storageMemoryUsed < storageRegionSize, +// s"bad test: storage memory used should be within the storage region") // Execution cannot evict storage because the latter is within the storage fraction, // so grant only what's remaining without evicting anything, i.e. 1000 - 300 - 400 = 300 assert(mm.acquireOnHeapExecutionMemory(400L, taskAttemptId) === 300L) @@ -172,10 +174,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes // First, ensure the test classes are set up as expected val expectedStorageRegionSize = 500L val expectedExecutionRegionSize = 500L - val storageRegionSize = getStorageRegionSize(mm) +// val storageRegionSize = getStorageRegionSize(mm) val executionRegionSize = maxMemory - expectedStorageRegionSize - require(storageRegionSize === expectedStorageRegionSize, - "bad test: storage region size is unexpected") +// require(storageRegionSize === expectedStorageRegionSize, +// "bad test: storage region size is unexpected") require(executionRegionSize === expectedExecutionRegionSize, "bad test: storage region size is unexpected") // Acquire enough execution memory to exceed the execution region From 82fffabad5a85a7bffee8bce953a16f82a53484c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 29 Oct 2015 13:33:15 -0700 Subject: [PATCH 05/31] Fix scalstyle. --- .../org/apache/spark/memory/StaticMemoryManagerSuite.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala index 3ba73e25d1842..a2f82e516900a 100644 --- a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala @@ -35,7 +35,10 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { maxExecutionMem: Long, maxStorageMem: Long): (StaticMemoryManager, MemoryStore) = { val mm = new StaticMemoryManager( - conf, maxOnHeapExecutionMemory = maxExecutionMem, maxStorageMemory = maxStorageMem, numCores = 1) + conf, + maxOnHeapExecutionMemory = maxExecutionMem, + maxStorageMemory = maxStorageMem, + numCores = 1) val ms = makeMemoryStore(mm) (mm, ms) } From b59dab960d6081aaa8d558d9d563ddcb445b9e4d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 1 Nov 2015 01:42:46 -0700 Subject: [PATCH 06/31] Attempt at fixing merge conflicts; refactor to use MemoryMode. --- .../apache/spark/memory/MemoryConsumer.java | 43 ++++++---- .../org/apache/spark/memory/MemoryMode.java | 26 ++++++ .../spark/memory/TaskMemoryManager.java | 86 +++++++++---------- .../apache/spark/memory/MemoryManager.scala | 76 ++++++---------- .../spark/memory/StaticMemoryManager.scala | 6 -- .../spark/memory/UnifiedMemoryManager.scala | 37 +++++--- .../spark/util/collection/Spillable.scala | 8 +- .../spark/memory/TaskMemoryManagerSuite.java | 34 ++++---- .../spark/memory/MemoryManagerSuite.scala | 60 ++++++------- .../memory/StaticMemoryManagerSuite.scala | 20 ++--- .../spark/memory/TestMemoryManager.scala | 27 +++--- .../memory/UnifiedMemoryManagerSuite.scala | 27 +++--- .../sql/execution/UnsafeKVExternalSorter.java | 4 +- 13 files changed, 237 insertions(+), 217 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/memory/MemoryMode.java diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 008799cc77395..94a9d6a91c61a 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -17,25 +17,23 @@ package org.apache.spark.memory; - import java.io.IOException; import org.apache.spark.unsafe.memory.MemoryBlock; - /** * An memory consumer of TaskMemoryManager, which support spilling. */ public abstract class MemoryConsumer { - private final TaskMemoryManager taskMemoryManager; + protected final TaskMemoryManager taskMemoryManager; private final long pageSize; - private long used; + private long onHeapMemoryUsed = 0L; + private long offHeapMemoryUsed = 0L; protected MemoryConsumer(TaskMemoryManager taskMemoryManager, long pageSize) { this.taskMemoryManager = taskMemoryManager; this.pageSize = pageSize; - this.used = 0; } protected MemoryConsumer(TaskMemoryManager taskMemoryManager) { @@ -43,10 +41,14 @@ protected MemoryConsumer(TaskMemoryManager taskMemoryManager) { } /** - * Returns the size of used memory in bytes. + * Returns the size of used on-heap memory in bytes. */ - long getUsed() { - return used; + long getMemoryUsed(MemoryMode mode) { + if (mode == MemoryMode.ON_HEAP) { + return onHeapMemoryUsed; + } else { + return offHeapMemoryUsed; + } } /** @@ -71,29 +73,30 @@ public void spill() throws IOException { * @return the amount of released memory in bytes * @throws IOException */ + // TODO(josh): clarify assumption that this only frees Tungsten-managed pages (for now). public abstract long spill(long size, MemoryConsumer trigger) throws IOException; /** - * Acquire `size` bytes memory. + * Acquire `size` bytes of on-heap execution memory. * * If there is not enough memory, throws OutOfMemoryError. */ protected void acquireMemory(long size) { - long got = taskMemoryManager.acquireExecutionMemory(size, this); + long got = taskMemoryManager.acquireExecutionMemory(size, MemoryMode.ON_HEAP, this); if (got < size) { - taskMemoryManager.releaseExecutionMemory(got, this); + taskMemoryManager.releaseExecutionMemory(got, MemoryMode.ON_HEAP, this); taskMemoryManager.showMemoryUsage(); throw new OutOfMemoryError("Could not acquire " + size + " bytes of memory, got " + got); } - used += got; + onHeapMemoryUsed += got; } /** * Release `size` bytes memory. */ protected void releaseMemory(long size) { - used -= size; - taskMemoryManager.releaseExecutionMemory(size, this); + onHeapMemoryUsed -= size; + taskMemoryManager.releaseExecutionMemory(size, MemoryMode.ON_HEAP, this); } /** @@ -114,7 +117,11 @@ protected MemoryBlock allocatePage(long required) { taskMemoryManager.showMemoryUsage(); throw new OutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got); } - used += page.size(); + if (taskMemoryManager.tungstenMemoryMode == MemoryMode.ON_HEAP) { + onHeapMemoryUsed += page.size(); + } else { + offHeapMemoryUsed += page.size(); + } return page; } @@ -122,7 +129,11 @@ protected MemoryBlock allocatePage(long required) { * Free a memory block. */ protected void freePage(MemoryBlock page) { - used -= page.size(); + if (taskMemoryManager.tungstenMemoryMode == MemoryMode.ON_HEAP) { + onHeapMemoryUsed -= page.size(); + } else { + offHeapMemoryUsed -= page.size(); + } taskMemoryManager.freePage(page, this); } } diff --git a/core/src/main/java/org/apache/spark/memory/MemoryMode.java b/core/src/main/java/org/apache/spark/memory/MemoryMode.java new file mode 100644 index 0000000000000..3a5e72d8aaec0 --- /dev/null +++ b/core/src/main/java/org/apache/spark/memory/MemoryMode.java @@ -0,0 +1,26 @@ +/* + * 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.memory; + +import org.apache.spark.annotation.Private; + +@Private +public enum MemoryMode { + ON_HEAP, + OFF_HEAP +} diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index e8c81ed923ae2..398db8fd56e06 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -103,7 +103,7 @@ public class TaskMemoryManager { * without doing any masking or lookups. Since this branching should be well-predicted by the JIT, * this extra layer of indirection / abstraction hopefully shouldn't be too expensive. */ - private final boolean inHeap; + final MemoryMode tungstenMemoryMode; /** * The size of memory granted to each consumer. @@ -115,7 +115,7 @@ public class TaskMemoryManager { * Construct a new TaskMemoryManager. */ public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) { - this.inHeap = memoryManager.tungstenMemoryIsAllocatedInHeap(); + this.tungstenMemoryMode = memoryManager.tungstenMemoryMode(); this.memoryManager = memoryManager; this.taskAttemptId = taskAttemptId; this.consumers = new HashSet<>(); @@ -127,23 +127,33 @@ public TaskMemoryManager(MemoryManager memoryManager, long taskAttemptId) { * * @return number of bytes successfully granted (<= N). */ - public long acquireExecutionMemory(long required, MemoryConsumer consumer) { + public long acquireExecutionMemory( + long required, + MemoryMode mode, + MemoryConsumer consumer) { assert(required >= 0); + // TODO(josh): handle spill differently based on type of request (on-heap vs off-heap). + // If we are allocating tungsten pages off-heap and receive a request to allocate on-heap + // memory here, then it may not make sense to spill since that would only end up freeing + // off-heap memory. This is subject to change, though, so it may be risky to make this + // optimization now in case we forget to undo it late when making changes. synchronized (this) { - long got = memoryManager.acquireExecutionMemory(required, taskAttemptId); + long got = memoryManager.acquireExecutionMemory(required, taskAttemptId, mode); // try to release memory from other consumers first, then we can reduce the frequency of // spilling, avoid to have too many spilled files. if (got < required) { // Call spill() on other consumers to release memory for (MemoryConsumer c: consumers) { - if (c != null && c != consumer && c.getUsed() > 0) { + if (c != null && c != consumer && c.getMemoryUsed(mode) > 0) { try { + // TODO(josh): subtlety / implementation detail: today, spill() happens to only + // release Tungsten pages. long released = c.spill(required - got, consumer); - if (released > 0) { + if (released > 0 && mode == tungstenMemoryMode) { logger.info("Task {} released {} from {} for {}", taskAttemptId, Utils.bytesToString(released), c, consumer); - got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId); + got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); if (got >= required) { break; } @@ -161,10 +171,10 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) { if (got < required && consumer != null) { try { long released = consumer.spill(required - got, consumer); - if (released > 0) { + if (released > 0 && mode == tungstenMemoryMode) { logger.info("Task {} released {} from itself ({})", taskAttemptId, Utils.bytesToString(released), consumer); - got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId); + got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); } } catch (IOException e) { logger.error("error while calling spill() on " + consumer, e); @@ -182,9 +192,9 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) { /** * Release N bytes of execution memory for a MemoryConsumer. */ - public void releaseExecutionMemory(long size, MemoryConsumer consumer) { + public void releaseExecutionMemory(long size, MemoryMode mode, MemoryConsumer consumer) { logger.debug("Task {} release {} from {}", taskAttemptId, Utils.bytesToString(size), consumer); - memoryManager.releaseExecutionMemory(size, taskAttemptId); + memoryManager.releaseExecutionMemory(size, taskAttemptId, mode); } /** @@ -194,8 +204,10 @@ public void showMemoryUsage() { logger.info("Memory used in task " + taskAttemptId); synchronized (this) { for (MemoryConsumer c: consumers) { - if (c.getUsed() > 0) { - logger.info("Acquired by " + c + ": " + Utils.bytesToString(c.getUsed())); + long totalMemUsage = + c.getMemoryUsed(MemoryMode.OFF_HEAP) + c.getMemoryUsed(MemoryMode.ON_HEAP); + if (totalMemUsage > 0) { + logger.info("Acquired by " + c + ": " + Utils.bytesToString(totalMemUsage)); } } } @@ -212,7 +224,8 @@ public long pageSizeBytes() { * Allocate a block of memory that will be tracked in the MemoryManager's page table; this is * intended for allocating large blocks of Tungsten memory that will be shared between operators. * - * Returns `null` if there was not enough memory to allocate the page. + * Returns `null` if there was not enough memory to allocate the page. May return a page that + * contains fewer bytes than requested, so callers should verify the size of returned pages. */ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { if (size > MAXIMUM_PAGE_SIZE_BYTES) { @@ -220,7 +233,7 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { "Cannot allocate a page with more than " + MAXIMUM_PAGE_SIZE_BYTES + " bytes"); } - long acquired = acquireExecutionMemory(size, consumer); + long acquired = acquireExecutionMemory(size, tungstenMemoryMode, consumer); if (acquired <= 0) { return null; } @@ -229,29 +242,12 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { synchronized (this) { pageNumber = allocatedPages.nextClearBit(0); if (pageNumber >= PAGE_TABLE_SIZE) { - releaseExecutionMemory(acquired, consumer); + releaseExecutionMemory(acquired, tungstenMemoryMode, consumer); throw new IllegalStateException( "Have already allocated a maximum of " + PAGE_TABLE_SIZE + " pages"); } allocatedPages.set(pageNumber); } - final long acquiredExecutionMemory; - if (memoryManager.tungstenMemoryIsAllocatedInHeap()) { - acquiredExecutionMemory = acquireOnHeapExecutionMemory(size); - } else { - acquiredExecutionMemory = acquireOffHeapExecutionMemory(size); - } - if (acquiredExecutionMemory != size) { - if (memoryManager.tungstenMemoryIsAllocatedInHeap()) { - releaseOnHeapExecutionMemory(acquiredExecutionMemory); - } else { - releaseOffHeapExecutionMemory(acquiredExecutionMemory); - } - synchronized (this) { - allocatedPages.clear(pageNumber); - } - return null; - } final MemoryBlock page = memoryManager.tungstenMemoryAllocator().allocate(size); page.pageNumber = pageNumber; pageTable[pageNumber] = page; @@ -275,13 +271,7 @@ public void freePage(MemoryBlock page, MemoryConsumer consumer) { if (logger.isTraceEnabled()) { logger.trace("Freed page number {} ({} bytes)", page.pageNumber, page.size()); } - long pageSize = page.size(); - memoryManager.tungstenMemoryAllocator().free(page); - if (memoryManager.tungstenMemoryIsAllocatedInHeap()) { - releaseOnHeapExecutionMemory(pageSize); - } else { - releaseOffHeapExecutionMemory(pageSize); - } + consumer.freePage(page); } /** @@ -295,7 +285,7 @@ public void freePage(MemoryBlock page, MemoryConsumer consumer) { * @return an encoded page address. */ public long encodePageNumberAndOffset(MemoryBlock page, long offsetInPage) { - if (!inHeap) { + if (tungstenMemoryMode == MemoryMode.OFF_HEAP) { // In off-heap mode, an offset is an absolute address that may require a full 64 bits to // encode. Due to our page size limitation, though, we can convert this into an offset that's // relative to the page's base offset; this relative offset will fit in 51 bits. @@ -324,7 +314,7 @@ private static long decodeOffset(long pagePlusOffsetAddress) { * {@link TaskMemoryManager#encodePageNumberAndOffset(MemoryBlock, long)} */ public Object getPage(long pagePlusOffsetAddress) { - if (inHeap) { + if (tungstenMemoryMode == MemoryMode.ON_HEAP) { final int pageNumber = decodePageNumber(pagePlusOffsetAddress); assert (pageNumber >= 0 && pageNumber < PAGE_TABLE_SIZE); final MemoryBlock page = pageTable[pageNumber]; @@ -342,7 +332,7 @@ public Object getPage(long pagePlusOffsetAddress) { */ public long getOffsetInPage(long pagePlusOffsetAddress) { final long offsetInPage = decodeOffset(pagePlusOffsetAddress); - if (inHeap) { + if (tungstenMemoryMode == MemoryMode.ON_HEAP) { return offsetInPage; } else { // In off-heap mode, an offset is an absolute address. In encodePageNumberAndOffset, we @@ -363,9 +353,15 @@ public long cleanUpAllAllocatedMemory() { synchronized (this) { Arrays.fill(pageTable, null); for (MemoryConsumer c: consumers) { - if (c != null && c.getUsed() > 0) { + if (c != null && c.getMemoryUsed(MemoryMode.ON_HEAP) > 0) { + // In case of failed task, it's normal to see leaked memory + logger.warn("leak " + Utils.bytesToString(c.getMemoryUsed(MemoryMode.ON_HEAP)) + + " of on-heap memory from " + c); + } + if (c != null && c.getMemoryUsed(MemoryMode.OFF_HEAP) > 0) { // In case of failed task, it's normal to see leaked memory - logger.warn("leak " + Utils.bytesToString(c.getUsed()) + " memory from " + c); + logger.warn("leak " + Utils.bytesToString(c.getMemoryUsed(MemoryMode.OFF_HEAP)) + + " of off-heap memory from " + c); } } consumers.clear(); diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index b7d1eb3b6f824..385e1b46a4b4b 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -19,10 +19,7 @@ package org.apache.spark.memory import scala.collection.mutable -import com.google.common.annotations.VisibleForTesting - -import org.apache.spark.util.Utils -import org.apache.spark.{SparkException, TaskContext, SparkConf, Logging} +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore} import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.memory.MemoryAllocator @@ -93,7 +90,7 @@ private[spark] abstract class MemoryManager( evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean /** - * Try to acquire up to `numBytes` of on-heap execution memory for the current task and return the + * Try to acquire up to `numBytes` of execution memory for the current task 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 @@ -102,54 +99,28 @@ private[spark] abstract class MemoryManager( * but an older task had a lot of memory already. */ private[memory] - def acquireOnHeapExecutionMemory(numBytes: Long, taskAttemptId: Long): Long - - /** - * Try to acquire up to `numBytes` of off-heap execution memory for the current task 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 - * task has a chance to ramp up to at least 1 / 2N of the total memory pool (where N is the # of - * active tasks) before it is forced to spill. This can happen if the number of tasks increase - * but an older task had a lot of memory already. - */ - private[memory] - def acquireOffHeapExecutionMemory( + def acquireExecutionMemory( numBytes: Long, - taskAttemptId: Long): Long = synchronized { - offHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) - } - - /** - * Release numBytes of on-heap execution memory belonging to the given task. - */ - private[memory] - def releaseOnHeapExecutionMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized { - onHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId) + taskAttemptId: Long, + memoryMode: MemoryMode): Long = synchronized { + memoryMode match { + case MemoryMode.ON_HEAP => onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) + case MemoryMode.OFF_HEAP => offHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) + } } /** - * Release numBytes of off-heap execution memory belonging to the given task. + * Release numBytes of execution memory belonging to the given task. */ private[memory] - final def releaseExecutionMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized { - val curMem = executionMemoryForTask.getOrElse(taskAttemptId, 0L) - if (curMem < numBytes) { - if (Utils.isTesting) { - throw new SparkException( - s"Internal error: release called on $numBytes bytes but task only has $curMem") - } else { - logWarning(s"Internal error: release called on $numBytes bytes but task only has $curMem") - } - } - if (executionMemoryForTask.contains(taskAttemptId)) { - executionMemoryForTask(taskAttemptId) -= numBytes - if (executionMemoryForTask(taskAttemptId) <= 0) { - executionMemoryForTask.remove(taskAttemptId) - } - releaseExecutionMemory(numBytes) + def releaseExecutionMemory( + numBytes: Long, + taskAttemptId: Long, + memoryMode: MemoryMode): Unit = synchronized { + memoryMode match { + case MemoryMode.ON_HEAP => onHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId) + case MemoryMode.OFF_HEAP => offHeapExecutionMemoryPool.releaseMemory(numBytes, taskAttemptId) } - notifyAll() // Notify waiters in acquireExecutionMemory() that memory has been freed } /** @@ -228,12 +199,17 @@ private[spark] abstract class MemoryManager( * Tracks whether Tungsten memory will be allocated on the JVM heap or off-heap using * sun.misc.Unsafe. */ - final val tungstenMemoryIsAllocatedInHeap: Boolean = - !conf.getBoolean("spark.unsafe.offHeap", false) + final val tungstenMemoryMode: MemoryMode = { + if (conf.getBoolean("spark.unsafe.offHeap", false)) MemoryMode.OFF_HEAP else MemoryMode.ON_HEAP + } /** * Allocates memory for use by Unsafe/Tungsten code. */ - private[memory] final val tungstenMemoryAllocator: MemoryAllocator = - if (tungstenMemoryIsAllocatedInHeap) MemoryAllocator.HEAP else MemoryAllocator.UNSAFE + private[memory] final val tungstenMemoryAllocator: MemoryAllocator = { + tungstenMemoryMode match { + case MemoryMode.ON_HEAP => MemoryAllocator.HEAP + case MemoryMode.OFF_HEAP => MemoryAllocator.UNSAFE + } + } } diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala index 054c676a1c8c5..dfc076b51e19e 100644 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala @@ -52,12 +52,6 @@ private[spark] class StaticMemoryManager( (maxStorageMemory * conf.getDouble("spark.storage.unrollFraction", 0.2)).toLong } - override private[memory] def acquireOnHeapExecutionMemory( - numBytes: Long, - taskAttemptId: Long): Long = synchronized { - onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) - } - override def acquireUnrollMemory( blockId: BlockId, numBytes: Long, diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index ca6bb9a67ba0a..6ea967833145b 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -62,19 +62,36 @@ private[spark] class UnifiedMemoryManager private[memory] ( maxMemory - onHeapExecutionMemoryPool.memoryUsed } - private[memory] def acquireOnHeapExecutionMemory( + + /** + * Try to acquire up to `numBytes` of execution memory for the current task 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 + * task has a chance to ramp up to at least 1 / 2N of the total memory pool (where N is the # of + * active tasks) before it is forced to spill. This can happen if the number of tasks increase + * but an older task had a lot of memory already. + */ + override private[memory] def acquireExecutionMemory( numBytes: Long, - taskAttemptId: Long): Long = synchronized { + taskAttemptId: Long, + memoryMode: MemoryMode): Long = synchronized { assert(numBytes >= 0) - val memoryBorrowedByStorage = math.max(0, storageMemoryPool.memoryUsed - minimumStoragePoolSize) - // If there is not enough free memory AND storage has borrowed some execution memory, - // then evict as much memory borrowed by storage as needed to grant this request - if (numBytes > onHeapExecutionMemoryPool.memoryFree && memoryBorrowedByStorage > 0) { - val spaceReclaimed = - storageMemoryPool.shrinkPoolByEvictingBlocks(math.min(numBytes, memoryBorrowedByStorage)) - onHeapExecutionMemoryPool.incrementPoolSize(spaceReclaimed) + memoryMode match { + case MemoryMode.ON_HEAP => + val memoryBorrowedByStorage = + math.max(0, storageMemoryPool.memoryUsed - minimumStoragePoolSize) + // If there is not enough free memory AND storage has borrowed some execution memory, + // then evict as much memory borrowed by storage as needed to grant this request + if (numBytes > onHeapExecutionMemoryPool.memoryFree && memoryBorrowedByStorage > 0) { + val spaceReclaimed = storageMemoryPool.shrinkPoolByEvictingBlocks( + math.min(numBytes, memoryBorrowedByStorage)) + onHeapExecutionMemoryPool.incrementPoolSize(spaceReclaimed) + } + onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) + case MemoryMode.OFF_HEAP => + super.acquireExecutionMemory(numBytes, taskAttemptId, memoryMode) } - onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) } override def acquireStorageMemory( diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index 9e002621a6909..3a48af82b1dae 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -17,7 +17,7 @@ package org.apache.spark.util.collection -import org.apache.spark.memory.TaskMemoryManager +import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} import org.apache.spark.{Logging, SparkEnv} /** @@ -78,7 +78,8 @@ private[spark] trait Spillable[C] extends Logging { if (elementsRead % 32 == 0 && currentMemory >= myMemoryThreshold) { // Claim up to double our current memory from the shuffle memory pool val amountToRequest = 2 * currentMemory - myMemoryThreshold - val granted = taskMemoryManager.acquireExecutionMemory(amountToRequest, null) + val granted = + taskMemoryManager.acquireExecutionMemory(amountToRequest, MemoryMode.ON_HEAP, null) myMemoryThreshold += granted // If 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 @@ -107,7 +108,8 @@ private[spark] trait Spillable[C] extends Logging { */ def releaseMemory(): Unit = { // The amount we requested does not include the initial memory tracking threshold - taskMemoryManager.releaseExecutionMemory(myMemoryThreshold - initialMemoryThreshold, null) + taskMemoryManager.releaseExecutionMemory( + myMemoryThreshold - initialMemoryThreshold, MemoryMode.ON_HEAP, null) myMemoryThreshold = initialMemoryThreshold } diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index dab7b0592cb4e..f65b2e56eee70 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -34,7 +34,7 @@ class TestMemoryConsumer extends MemoryConsumer { @Override public long spill(long size, MemoryConsumer trigger) throws IOException { - long used = getUsed(); + long used = getMemoryUsed(taskMemoryManager.tungstenMemoryMode); releaseMemory(used); return used; } @@ -88,33 +88,33 @@ public void cooperativeSpilling() { TestMemoryConsumer c1 = new TestMemoryConsumer(manager); TestMemoryConsumer c2 = new TestMemoryConsumer(manager); c1.use(100); - assert(c1.getUsed() == 100); + assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 100); c2.use(100); - assert(c2.getUsed() == 100); - assert(c1.getUsed() == 0); // spilled + assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 100); + assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 0); // spilled c1.use(100); - assert(c1.getUsed() == 100); - assert(c2.getUsed() == 0); // spilled + assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 100); + assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 0); // spilled c1.use(50); - assert(c1.getUsed() == 50); // spilled - assert(c2.getUsed() == 0); + assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 50); // spilled + assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 0); c2.use(50); - assert(c1.getUsed() == 50); - assert(c2.getUsed() == 50); + assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 50); + assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 50); c1.use(100); - assert(c1.getUsed() == 100); - assert(c2.getUsed() == 0); // spilled + assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 100); + assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 0); // spilled c1.free(20); - assert(c1.getUsed() == 80); + assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 80); c2.use(10); - assert(c1.getUsed() == 80); - assert(c2.getUsed() == 10); + assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 80); + assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 10); c2.use(100); - assert(c2.getUsed() == 100); - assert(c1.getUsed() == 0); // spilled + assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 100); + assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 0); // spilled c1.free(0); c2.free(100); diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 4a9479cf490fb..923ececd83888 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -145,20 +145,20 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val manager = createMemoryManager(1000L) val taskMemoryManager = new TaskMemoryManager(manager, 0) - assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 100L) - assert(taskMemoryManager.acquireExecutionMemory(400L, null) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(400L, null) === 400L) - assert(taskMemoryManager.acquireExecutionMemory(200L, null) === 100L) - assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 0L) - assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 100L) + assert(taskMemoryManager.acquireExecutionMemory(400L, MemoryMode.ON_HEAP, null) === 400L) + assert(taskMemoryManager.acquireExecutionMemory(400L, MemoryMode.ON_HEAP, null) === 400L) + assert(taskMemoryManager.acquireExecutionMemory(200L, MemoryMode.ON_HEAP, null) === 100L) + assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) - taskMemoryManager.releaseExecutionMemory(500L, null) - assert(taskMemoryManager.acquireExecutionMemory(300L, null) === 300L) - assert(taskMemoryManager.acquireExecutionMemory(300L, null) === 200L) + taskMemoryManager.releaseExecutionMemory(500L, MemoryMode.ON_HEAP, null) + assert(taskMemoryManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) === 300L) + assert(taskMemoryManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) === 200L) taskMemoryManager.cleanUpAllAllocatedMemory() - assert(taskMemoryManager.acquireExecutionMemory(1000L, null) === 1000L) - assert(taskMemoryManager.acquireExecutionMemory(100L, null) === 0L) + assert(taskMemoryManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) === 1000L) + assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) } test("two tasks requesting full execution memory") { @@ -168,15 +168,15 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val futureTimeout: Duration = 20.seconds // Have both tasks request 500 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, null) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } assert(Await.result(t1Result1, futureTimeout) === 500L) assert(Await.result(t2Result1, futureTimeout) === 500L) // Have both tasks each request 500 bytes more; both should immediately return 0 as they are // both now at 1 / N - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, null) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } assert(Await.result(t1Result2, 200.millis) === 0L) assert(Await.result(t2Result2, 200.millis) === 0L) } @@ -188,15 +188,15 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val futureTimeout: Duration = 20.seconds // Have both tasks request 250 bytes, then wait until both requests have been granted: - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, null) } - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } assert(Await.result(t1Result1, futureTimeout) === 250L) assert(Await.result(t2Result1, futureTimeout) === 250L) // Have both tasks each request 500 bytes more. // We should only grant 250 bytes to each of them on this second request - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, null) } - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } assert(Await.result(t1Result2, futureTimeout) === 250L) assert(Await.result(t2Result2, futureTimeout) === 250L) } @@ -208,17 +208,17 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) } assert(Await.result(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) - t1MemManager.releaseExecutionMemory(250L, null) + t1MemManager.releaseExecutionMemory(250L, MemoryMode.ON_HEAP, null) // The memory freed from t1 should now be granted to t2. assert(Await.result(t2Result1, futureTimeout) === 250L) // Further requests by t2 should be denied immediately because it now has 1 / 2N of the memory. - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, null) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) } assert(Await.result(t2Result2, 200.millis) === 0L) } @@ -229,18 +229,18 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val futureTimeout: Duration = 20.seconds // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) } assert(Await.result(t1Result1, futureTimeout) === 1000L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) // t1 releases all of its memory, so t2 should be able to grab all of the memory t1MemManager.cleanUpAllAllocatedMemory() assert(Await.result(t2Result1, futureTimeout) === 500L) - val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, null) } + val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } assert(Await.result(t2Result2, futureTimeout) === 500L) - val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, null) } + val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } assert(Await.result(t2Result3, 200.millis) === 0L) } @@ -251,13 +251,13 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val t2MemManager = new TaskMemoryManager(memoryManager, 2) val futureTimeout: Duration = 20.seconds - val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, null) } + val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, MemoryMode.ON_HEAP, null) } assert(Await.result(t1Result1, futureTimeout) === 700L) - val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, null) } + val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) } assert(Await.result(t2Result1, futureTimeout) === 300L) - val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, null) } + val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) } assert(Await.result(t1Result2, 200.millis) === 0L) } } diff --git a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala index a2f82e516900a..d442bf4c8462c 100644 --- a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala @@ -56,21 +56,21 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { val taskAttemptId = 0L val (mm, _) = makeThings(maxExecutionMem, Long.MaxValue) assert(mm.executionMemoryUsed === 0L) - assert(mm.acquireOnHeapExecutionMemory(10L, taskAttemptId) === 10L) + assert(mm.acquireExecutionMemory(10L, taskAttemptId, MemoryMode.ON_HEAP) === 10L) assert(mm.executionMemoryUsed === 10L) - assert(mm.acquireOnHeapExecutionMemory(100L, taskAttemptId) === 100L) + assert(mm.acquireExecutionMemory(100L, taskAttemptId, MemoryMode.ON_HEAP) === 100L) // Acquire up to the max - assert(mm.acquireOnHeapExecutionMemory(1000L, taskAttemptId) === 890L) + assert(mm.acquireExecutionMemory(1000L, taskAttemptId, MemoryMode.ON_HEAP) === 890L) assert(mm.executionMemoryUsed === maxExecutionMem) - assert(mm.acquireOnHeapExecutionMemory(1L, taskAttemptId) === 0L) + assert(mm.acquireExecutionMemory(1L, taskAttemptId, MemoryMode.ON_HEAP) === 0L) assert(mm.executionMemoryUsed === maxExecutionMem) - mm.releaseOnHeapExecutionMemory(800L, taskAttemptId) + mm.releaseExecutionMemory(800L, taskAttemptId, MemoryMode.ON_HEAP) assert(mm.executionMemoryUsed === 200L) // Acquire after release - assert(mm.acquireOnHeapExecutionMemory(1L, taskAttemptId) === 1L) + assert(mm.acquireExecutionMemory(1L, taskAttemptId, MemoryMode.ON_HEAP) === 1L) assert(mm.executionMemoryUsed === 201L) // Release beyond what was acquired - mm.releaseOnHeapExecutionMemory(maxExecutionMem, taskAttemptId) + mm.releaseExecutionMemory(maxExecutionMem, taskAttemptId, MemoryMode.ON_HEAP) assert(mm.executionMemoryUsed === 0L) } @@ -120,10 +120,10 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { val dummyBlock = TestBlockId("ain't nobody love like you do") val (mm, ms) = makeThings(maxExecutionMem, maxStorageMem) // Only execution memory should increase - assert(mm.acquireOnHeapExecutionMemory(100L, taskAttemptId) === 100L) + assert(mm.acquireExecutionMemory(100L, taskAttemptId, MemoryMode.ON_HEAP) === 100L) assert(mm.storageMemoryUsed === 0L) assert(mm.executionMemoryUsed === 100L) - assert(mm.acquireOnHeapExecutionMemory(1000L, taskAttemptId) === 100L) + assert(mm.acquireExecutionMemory(1000L, taskAttemptId, MemoryMode.ON_HEAP) === 100L) assert(mm.storageMemoryUsed === 0L) assert(mm.executionMemoryUsed === 200L) // Only storage memory should increase @@ -132,7 +132,7 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { assert(mm.storageMemoryUsed === 50L) assert(mm.executionMemoryUsed === 200L) // Only execution memory should be released - mm.releaseOnHeapExecutionMemory(133L, taskAttemptId) + mm.releaseExecutionMemory(133L, taskAttemptId, MemoryMode.ON_HEAP) assert(mm.storageMemoryUsed === 50L) assert(mm.executionMemoryUsed === 67L) // Only storage memory should be released diff --git a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala index 676f5fcb876d2..0ae95541a08e8 100644 --- a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala +++ b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala @@ -22,10 +22,15 @@ import scala.collection.mutable import org.apache.spark.SparkConf import org.apache.spark.storage.{BlockStatus, BlockId} -class TestMemoryManager(conf: SparkConf) extends MemoryManager(conf, numCores = 1) { - private[memory] override def doAcquireExecutionMemory( +class TestMemoryManager(conf: SparkConf) + extends MemoryManager(conf, numCores = 1, maxOnHeapExecutionMemory = Long.MaxValue) { + + // TODO(josh): separate configs for available on- and off-heap + + override private[memory] def acquireExecutionMemory( numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Long = synchronized { + taskAttemptId: Long, + memoryMode: MemoryMode): Long = { if (oomOnce) { oomOnce = false 0 @@ -33,17 +38,11 @@ class TestMemoryManager(conf: SparkConf) extends MemoryManager(conf, numCores = available -= numBytes numBytes } else { - _executionMemoryUsed += available val grant = available available = 0 grant } } - override private[memory] def acquireOffHeapExecutionMemory( - numBytes: Long, - taskAttemptId: Long): Long = { - acquireOnHeapExecutionMemory(numBytes, taskAttemptId) - } override def acquireStorageMemory( blockId: BlockId, numBytes: Long, @@ -52,13 +51,11 @@ class TestMemoryManager(conf: SparkConf) extends MemoryManager(conf, numCores = blockId: BlockId, numBytes: Long, evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = true - override def releaseExecutionMemory(numBytes: Long): Unit = { - available += numBytes - _executionMemoryUsed -= numBytes - } override def releaseStorageMemory(numBytes: Long): Unit = {} - override private[memory] def releaseOnHeapExecutionMemory(numBytes: Long, tid: Long): Unit = {} - override private[memory] def releaseOffHeapExecutionMemory(numBytes: Long, tid: Long): Unit = {} + override private[memory] def releaseExecutionMemory( + numBytes: Long, + taskAttemptId: Long, + memoryMode: MemoryMode): Unit = {} override def maxStorageMemory: Long = Long.MaxValue private var oomOnce = false diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 1fbd85b51caed..4d860310721f8 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -24,7 +24,6 @@ import org.scalatest.PrivateMethodTester import org.apache.spark.SparkConf import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore, TestBlockId} - class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTester { private val dummyBlock = TestBlockId("--") private val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -65,21 +64,21 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes val taskAttemptId = 0L val (mm, _) = makeThings(maxMemory) assert(mm.executionMemoryUsed === 0L) - assert(mm.acquireOnHeapExecutionMemory(10L, taskAttemptId) === 10L) + assert(mm.acquireExecutionMemory(10L, taskAttemptId, MemoryMode.ON_HEAP) === 10L) assert(mm.executionMemoryUsed === 10L) - assert(mm.acquireOnHeapExecutionMemory(100L, taskAttemptId) === 100L) + assert(mm.acquireExecutionMemory(100L, taskAttemptId, MemoryMode.ON_HEAP) === 100L) // Acquire up to the max - assert(mm.acquireOnHeapExecutionMemory(1000L, taskAttemptId) === 890L) + assert(mm.acquireExecutionMemory(1000L, taskAttemptId, MemoryMode.ON_HEAP) === 890L) assert(mm.executionMemoryUsed === maxMemory) - assert(mm.acquireOnHeapExecutionMemory(1L, taskAttemptId) === 0L) + assert(mm.acquireExecutionMemory(1L, taskAttemptId, MemoryMode.ON_HEAP) === 0L) assert(mm.executionMemoryUsed === maxMemory) - mm.releaseOnHeapExecutionMemory(800L, taskAttemptId) + mm.releaseExecutionMemory(800L, taskAttemptId, MemoryMode.ON_HEAP) assert(mm.executionMemoryUsed === 200L) // Acquire after release - assert(mm.acquireOnHeapExecutionMemory(1L, taskAttemptId) === 1L) + assert(mm.acquireExecutionMemory(1L, taskAttemptId, MemoryMode.ON_HEAP) === 1L) assert(mm.executionMemoryUsed === 201L) // Release beyond what was acquired - mm.releaseOnHeapExecutionMemory(maxMemory, taskAttemptId) + mm.releaseExecutionMemory(maxMemory, taskAttemptId, MemoryMode.ON_HEAP) assert(mm.executionMemoryUsed === 0L) } @@ -142,12 +141,12 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes // require(mm.storageMemoryUsed > storageRegionSize, // s"bad test: storage memory used should exceed the storage region") // Execution needs to request 250 bytes to evict storage memory - assert(mm.acquireOnHeapExecutionMemory(100L, taskAttemptId) === 100L) + assert(mm.acquireExecutionMemory(100L, taskAttemptId, MemoryMode.ON_HEAP) === 100L) assert(mm.executionMemoryUsed === 100L) assert(mm.storageMemoryUsed === 750L) assertEnsureFreeSpaceNotCalled(ms) // Execution wants 200 bytes but only 150 are free, so storage is evicted - assert(mm.acquireOnHeapExecutionMemory(200L, taskAttemptId) === 200L) + assert(mm.acquireExecutionMemory(200L, taskAttemptId, MemoryMode.ON_HEAP) === 200L) assertEnsureFreeSpaceCalled(ms, 200L) assert(mm.executionMemoryUsed === 300L) mm.releaseAllStorageMemory() @@ -161,7 +160,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes // s"bad test: storage memory used should be within the storage region") // Execution cannot evict storage because the latter is within the storage fraction, // so grant only what's remaining without evicting anything, i.e. 1000 - 300 - 400 = 300 - assert(mm.acquireOnHeapExecutionMemory(400L, taskAttemptId) === 300L) + assert(mm.acquireExecutionMemory(400L, taskAttemptId, MemoryMode.ON_HEAP) === 300L) assert(mm.executionMemoryUsed === 600L) assert(mm.storageMemoryUsed === 400L) assertEnsureFreeSpaceNotCalled(ms) @@ -181,7 +180,7 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes require(executionRegionSize === expectedExecutionRegionSize, "bad test: storage region size is unexpected") // Acquire enough execution memory to exceed the execution region - assert(mm.acquireOnHeapExecutionMemory(800L, taskAttemptId) === 800L) + assert(mm.acquireExecutionMemory(800L, taskAttemptId, MemoryMode.ON_HEAP) === 800L) assert(mm.executionMemoryUsed === 800L) assert(mm.storageMemoryUsed === 0L) assertEnsureFreeSpaceNotCalled(ms) @@ -196,10 +195,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes assert(mm.executionMemoryUsed === 800L) assert(mm.storageMemoryUsed === 100L) assertEnsureFreeSpaceCalled(ms, 250L) - mm.releaseOnHeapExecutionMemory(maxMemory, taskAttemptId) + mm.releaseExecutionMemory(maxMemory, taskAttemptId, MemoryMode.ON_HEAP) mm.releaseStorageMemory(maxMemory) // Acquire some execution memory again, but this time keep it within the execution region - assert(mm.acquireOnHeapExecutionMemory(200L, taskAttemptId) === 200L) + assert(mm.acquireExecutionMemory(200L, taskAttemptId, MemoryMode.ON_HEAP) === 200L) assert(mm.executionMemoryUsed === 200L) assert(mm.storageMemoryUsed === 0L) assertEnsureFreeSpaceNotCalled(ms) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 845f2ae6859b7..3d8cacf50198a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.spark.TaskContext; +import org.apache.spark.memory.MemoryMode; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; import org.apache.spark.sql.catalyst.expressions.codegen.BaseOrdering; @@ -126,7 +127,8 @@ public UnsafeKVExternalSorter( sorter.spill(); map.free(); // counting the memory used UnsafeInMemorySorter - taskMemoryManager.acquireExecutionMemory(inMemSorter.getMemoryUsage(), sorter); + taskMemoryManager.acquireExecutionMemory( + inMemSorter.getMemoryUsage(), MemoryMode.ON_HEAP, sorter); } } From 8bbc111b800a921d9143e747a3d2d4bee1db20b9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 1 Nov 2015 12:14:33 -0800 Subject: [PATCH 07/31] Fix double-free of pages. --- core/src/main/java/org/apache/spark/memory/MemoryConsumer.java | 2 +- .../main/java/org/apache/spark/memory/TaskMemoryManager.java | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 94a9d6a91c61a..b7b04280cd7db 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -134,6 +134,6 @@ protected void freePage(MemoryBlock page) { } else { offHeapMemoryUsed -= page.size(); } - taskMemoryManager.freePage(page, this); + taskMemoryManager.freePage(page); } } diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 398db8fd56e06..d0e9181bf876a 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -260,7 +260,7 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { /** * Free a block of memory allocated via {@link TaskMemoryManager#allocatePage}. */ - public void freePage(MemoryBlock page, MemoryConsumer consumer) { + public void freePage(MemoryBlock page) { assert (page.pageNumber != -1) : "Called freePage() on memory that wasn't allocated with allocatePage()"; assert(allocatedPages.get(page.pageNumber)); @@ -271,7 +271,6 @@ public void freePage(MemoryBlock page, MemoryConsumer consumer) { if (logger.isTraceEnabled()) { logger.trace("Freed page number {} ({} bytes)", page.pageNumber, page.size()); } - consumer.freePage(page); } /** From df2168f1a01f7229d14d7f742d678d75e192a5df Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 1 Nov 2015 13:28:13 -0800 Subject: [PATCH 08/31] Fixes to execution evicting storage. --- .../spark/memory/StorageMemoryPool.scala | 20 +++++++++++-------- .../spark/memory/UnifiedMemoryManager.scala | 16 ++++++++++----- .../spark/memory/MemoryManagerSuite.scala | 10 ++++++++-- .../memory/UnifiedMemoryManagerSuite.scala | 2 ++ 4 files changed, 33 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index 221a8e30e8616..ed7bc72b45d4a 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -72,7 +72,7 @@ class StorageMemoryPool extends MemoryPool with Logging { evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { assert(numBytesToAcquire >= 0) assert(numBytesToFree >= 0) - // TODO(josh): check whether there is enough memory / handle eviction + assert(memoryUsed <= poolSize) memoryStore.ensureFreeSpace(blockId, numBytesToFree, evictedBlocks) // Register evicted blocks, if any, with the active task metrics Option(TaskContext.get()).foreach { tc => @@ -80,8 +80,9 @@ class StorageMemoryPool extends MemoryPool with Logging { val lastUpdatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) metrics.updatedBlocks = Some(lastUpdatedBlocks ++ evictedBlocks.toSeq) } - // TODO(josh): is this assertion in the right place? - assert(memoryUsed <= poolSize) + // NOTE: If the memory store evicts blocks, then those evictions will synchronously call + // back into this StorageMemoryPool in order to free. Therefore, these variables should have + // been updated. val enoughMemory = numBytesToAcquire <= memoryFree if (enoughMemory) { _memoryUsed += numBytesToAcquire @@ -103,12 +104,15 @@ class StorageMemoryPool extends MemoryPool with Logging { _memoryUsed = 0 } + // TODO(josh): comment def shrinkPoolByEvictingBlocks(spaceToEnsure: Long): Long = synchronized { + val spaceFreedByReleasingUnusedMemory = Math.min(spaceToEnsure, memoryFree) + decrementPoolSize(spaceFreedByReleasingUnusedMemory) val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - memoryStore.ensureFreeSpace(spaceToEnsure, evictedBlocks) - val spaceFreed = evictedBlocks.map(_._2.memSize).sum - _memoryUsed -= spaceFreed - decrementPoolSize(spaceFreed) - spaceFreed + memoryStore.ensureFreeSpace(spaceToEnsure - spaceFreedByReleasingUnusedMemory, evictedBlocks) + val spaceFreedByEviction = evictedBlocks.map(_._2.memSize).sum + _memoryUsed -= spaceFreedByEviction + decrementPoolSize(spaceFreedByEviction) + spaceFreedByReleasingUnusedMemory + spaceFreedByEviction } } diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 6ea967833145b..5ecd2ec5773b6 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -54,8 +54,11 @@ private[spark] class UnifiedMemoryManager private[memory] ( extends MemoryManager( conf, numCores, + // TODO(josh): it is confusing how this interacts with page size calculations: maxOnHeapExecutionMemory = maxMemory - minimumStoragePoolSize) { + // At first, all memory is allocated towards execution. + // TODO(josh): in light of this policy, the name minimumStoragePoolSize is confusing. onHeapExecutionMemoryPool.incrementPoolSize(maxMemory) override def maxStorageMemory: Long = synchronized { @@ -76,11 +79,13 @@ private[spark] class UnifiedMemoryManager private[memory] ( numBytes: Long, taskAttemptId: Long, memoryMode: MemoryMode): Long = synchronized { + assert(onHeapExecutionMemoryPool.poolSize + storageMemoryPool.poolSize == maxMemory) assert(numBytes >= 0) memoryMode match { case MemoryMode.ON_HEAP => val memoryBorrowedByStorage = - math.max(0, storageMemoryPool.memoryUsed - minimumStoragePoolSize) + math.max(storageMemoryPool.memoryFree, + storageMemoryPool.poolSize - minimumStoragePoolSize) // If there is not enough free memory AND storage has borrowed some execution memory, // then evict as much memory borrowed by storage as needed to grant this request if (numBytes > onHeapExecutionMemoryPool.memoryFree && memoryBorrowedByStorage > 0) { @@ -98,10 +103,11 @@ private[spark] class UnifiedMemoryManager private[memory] ( blockId: BlockId, numBytes: Long, evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { - if (numBytes > storageMemoryPool.memoryFree - && numBytes <= onHeapExecutionMemoryPool.memoryFree) { - onHeapExecutionMemoryPool.decrementPoolSize(numBytes) - storageMemoryPool.incrementPoolSize(numBytes) + assert(onHeapExecutionMemoryPool.poolSize + storageMemoryPool.poolSize == maxMemory) + if (numBytes > storageMemoryPool.memoryFree) { + val memoryBorrowedFromExecution = Math.min(onHeapExecutionMemoryPool.memoryFree, numBytes) + onHeapExecutionMemoryPool.decrementPoolSize(memoryBorrowedFromExecution) + storageMemoryPool.incrementPoolSize(memoryBorrowedFromExecution) } storageMemoryPool.acquireMemory(blockId, numBytes, evictedBlocks) } diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 923ececd83888..544c50dbe408a 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.memory import java.util.concurrent.atomic.AtomicLong +import scala.collection.mutable import scala.concurrent.duration.Duration import scala.concurrent.{Await, ExecutionContext, Future} @@ -29,7 +30,7 @@ import org.mockito.stubbing.Answer import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkFunSuite -import org.apache.spark.storage.MemoryStore +import org.apache.spark.storage.{StorageLevel, BlockStatus, BlockId, MemoryStore} /** @@ -78,7 +79,12 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { require(args(numBytesPos).isInstanceOf[Long], s"bad test: expected ensureFreeSpace " + s"argument at index $numBytesPos to be a Long: ${args.mkString(", ")}") val numBytes = args(numBytesPos).asInstanceOf[Long] - mockEnsureFreeSpace(mm, numBytes) + val success = mockEnsureFreeSpace(mm, numBytes) + if (success) { + args.last.asInstanceOf[mutable.Buffer[(BlockId, BlockStatus)]].append( + (null, BlockStatus(StorageLevel.MEMORY_ONLY, numBytes, 0L, 0L))) + } + success } } } diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 4d860310721f8..8c15d322f8ce8 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -156,6 +156,8 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes // Acquire some storage memory again, but this time keep it within the storage region assert(mm.acquireStorageMemory(dummyBlock, 400L, evictedBlocks)) assertEnsureFreeSpaceCalled(ms, 400L) + assert(mm.storageMemoryUsed === 400L) + assert(mm.executionMemoryUsed === 300L) // require(mm.storageMemoryUsed < storageRegionSize, // s"bad test: storage memory used should be within the storage region") // Execution cannot evict storage because the latter is within the storage fraction, From 144e68042fb773295d69f57bd785cc1eeed4659c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 1 Nov 2015 13:31:22 -0800 Subject: [PATCH 09/31] Avoid unnecessary ensureFreeSpace() calls --- .../spark/memory/StorageMemoryPool.scala | 18 +++++++++++------- .../spark/memory/UnifiedMemoryManager.scala | 2 +- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index ed7bc72b45d4a..d05631bc4161f 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -105,14 +105,18 @@ class StorageMemoryPool extends MemoryPool with Logging { } // TODO(josh): comment - def shrinkPoolByEvictingBlocks(spaceToEnsure: Long): Long = synchronized { + def shrinkPoolToFreeSpace(spaceToEnsure: Long): Long = synchronized { val spaceFreedByReleasingUnusedMemory = Math.min(spaceToEnsure, memoryFree) decrementPoolSize(spaceFreedByReleasingUnusedMemory) - val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - memoryStore.ensureFreeSpace(spaceToEnsure - spaceFreedByReleasingUnusedMemory, evictedBlocks) - val spaceFreedByEviction = evictedBlocks.map(_._2.memSize).sum - _memoryUsed -= spaceFreedByEviction - decrementPoolSize(spaceFreedByEviction) - spaceFreedByReleasingUnusedMemory + spaceFreedByEviction + if (spaceFreedByReleasingUnusedMemory == spaceToEnsure) { + spaceFreedByReleasingUnusedMemory + } else { + val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + memoryStore.ensureFreeSpace(spaceToEnsure - spaceFreedByReleasingUnusedMemory, evictedBlocks) + val spaceFreedByEviction = evictedBlocks.map(_._2.memSize).sum + _memoryUsed -= spaceFreedByEviction + decrementPoolSize(spaceFreedByEviction) + spaceFreedByReleasingUnusedMemory + spaceFreedByEviction + } } } diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 5ecd2ec5773b6..26be295a32bad 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -89,7 +89,7 @@ private[spark] class UnifiedMemoryManager private[memory] ( // If there is not enough free memory AND storage has borrowed some execution memory, // then evict as much memory borrowed by storage as needed to grant this request if (numBytes > onHeapExecutionMemoryPool.memoryFree && memoryBorrowedByStorage > 0) { - val spaceReclaimed = storageMemoryPool.shrinkPoolByEvictingBlocks( + val spaceReclaimed = storageMemoryPool.shrinkPoolToFreeSpace( math.min(numBytes, memoryBorrowedByStorage)) onHeapExecutionMemoryPool.incrementPoolSize(spaceReclaimed) } From 542dd56e047023cb4d7e0d62815f2bc4cd92ae84 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 1 Nov 2015 17:55:56 -0800 Subject: [PATCH 10/31] Fix memory leak detection test in TaskMemoryManager. --- .../org/apache/spark/memory/TaskMemoryManager.java | 13 ++++++++----- .../apache/spark/memory/TaskMemoryManagerSuite.java | 8 +++++++- .../org/apache/spark/memory/TestMemoryManager.scala | 3 +++ 3 files changed, 18 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index d0e9181bf876a..95c5829578f8a 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -106,7 +106,7 @@ public class TaskMemoryManager { final MemoryMode tungstenMemoryMode; /** - * The size of memory granted to each consumer. + * Tracks spillable memory consumers. */ @GuardedBy("this") private final HashSet consumers; @@ -145,7 +145,7 @@ public long acquireExecutionMemory( if (got < required) { // Call spill() on other consumers to release memory for (MemoryConsumer c: consumers) { - if (c != null && c != consumer && c.getMemoryUsed(mode) > 0) { + if (c != consumer && c.getMemoryUsed(mode) > 0) { try { // TODO(josh): subtlety / implementation detail: today, spill() happens to only // release Tungsten pages. @@ -183,7 +183,9 @@ public long acquireExecutionMemory( } } - consumers.add(consumer); + if (consumer != null) { + consumers.add(consumer); + } logger.debug("Task {} acquire {} for {}", taskAttemptId, Utils.bytesToString(got), consumer); return got; } @@ -200,6 +202,7 @@ public void releaseExecutionMemory(long size, MemoryMode mode, MemoryConsumer co /** * Dump the memory usage of all consumers. */ + // TODO(josh): also report memory not allocated to any particular consumer. public void showMemoryUsage() { logger.info("Memory used in task " + taskAttemptId); synchronized (this) { @@ -352,12 +355,12 @@ public long cleanUpAllAllocatedMemory() { synchronized (this) { Arrays.fill(pageTable, null); for (MemoryConsumer c: consumers) { - if (c != null && c.getMemoryUsed(MemoryMode.ON_HEAP) > 0) { + if (c.getMemoryUsed(MemoryMode.ON_HEAP) > 0) { // In case of failed task, it's normal to see leaked memory logger.warn("leak " + Utils.bytesToString(c.getMemoryUsed(MemoryMode.ON_HEAP)) + " of on-heap memory from " + c); } - if (c != null && c.getMemoryUsed(MemoryMode.OFF_HEAP) > 0) { + if (c.getMemoryUsed(MemoryMode.OFF_HEAP) > 0) { // In case of failed task, it's normal to see leaked memory logger.warn("leak " + Utils.bytesToString(c.getMemoryUsed(MemoryMode.OFF_HEAP)) + " of off-heap memory from " + c); diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index f65b2e56eee70..c1e0f84bce2f5 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -51,8 +51,14 @@ void free(long size) { @Test public void leakedPageMemoryIsDetected() { final TaskMemoryManager manager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.unsafe.offHeap", "false")), 0); + new StaticMemoryManager( + new SparkConf().set("spark.unsafe.offHeap", "false"), + Long.MAX_VALUE, + Long.MAX_VALUE, + 1), + 0); manager.allocatePage(4096, null); // leak memory + Assert.assertEquals(4096, manager.getMemoryConsumptionForThisTask()); Assert.assertEquals(4096, manager.cleanUpAllAllocatedMemory()); } diff --git a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala index 0ae95541a08e8..9d552ba414033 100644 --- a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala +++ b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala @@ -25,6 +25,9 @@ import org.apache.spark.storage.{BlockStatus, BlockId} class TestMemoryManager(conf: SparkConf) extends MemoryManager(conf, numCores = 1, maxOnHeapExecutionMemory = Long.MaxValue) { + storageMemoryPool.incrementPoolSize(Long.MaxValue) + onHeapExecutionMemoryPool.incrementPoolSize(Long.MaxValue) + // TODO(josh): separate configs for available on- and off-heap override private[memory] def acquireExecutionMemory( From 709ecf2053e369c22a8b9fcc8baf6a41c4ab5b3f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 1 Nov 2015 18:02:12 -0800 Subject: [PATCH 11/31] Fix TaskMemoryManagerSuite. --- .../main/java/org/apache/spark/memory/TaskMemoryManager.java | 1 + .../scala/org/apache/spark/memory/TestMemoryManager.scala | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 95c5829578f8a..9096ac6fee710 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -368,6 +368,7 @@ public long cleanUpAllAllocatedMemory() { } consumers.clear(); } + return memoryManager.releaseAllExecutionMemoryForTask(taskAttemptId); } diff --git a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala index 9d552ba414033..51018d7066c75 100644 --- a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala +++ b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala @@ -58,7 +58,9 @@ class TestMemoryManager(conf: SparkConf) override private[memory] def releaseExecutionMemory( numBytes: Long, taskAttemptId: Long, - memoryMode: MemoryMode): Unit = {} + memoryMode: MemoryMode): Unit = { + available += numBytes + } override def maxStorageMemory: Long = Long.MaxValue private var oomOnce = false From 1356cdb145c8f060bfb718dac1005f5d5614e7a3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 1 Nov 2015 18:36:03 -0800 Subject: [PATCH 12/31] Another fix to freeing pages --- .../main/java/org/apache/spark/memory/MemoryConsumer.java | 2 +- .../main/java/org/apache/spark/memory/TaskMemoryManager.java | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index b7b04280cd7db..94a9d6a91c61a 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -134,6 +134,6 @@ protected void freePage(MemoryBlock page) { } else { offHeapMemoryUsed -= page.size(); } - taskMemoryManager.freePage(page); + taskMemoryManager.freePage(page, this); } } diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 9096ac6fee710..130e941e63003 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -263,7 +263,7 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { /** * Free a block of memory allocated via {@link TaskMemoryManager#allocatePage}. */ - public void freePage(MemoryBlock page) { + public void freePage(MemoryBlock page, MemoryConsumer consumer) { assert (page.pageNumber != -1) : "Called freePage() on memory that wasn't allocated with allocatePage()"; assert(allocatedPages.get(page.pageNumber)); @@ -274,6 +274,9 @@ public void freePage(MemoryBlock page) { if (logger.isTraceEnabled()) { logger.trace("Freed page number {} ({} bytes)", page.pageNumber, page.size()); } + long pageSize = page.size(); + memoryManager.tungstenMemoryAllocator().free(page); + releaseExecutionMemory(pageSize, tungstenMemoryMode, consumer); } /** From d8ffd35fa6dcaacc6cefdd4976750125800fa1e9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 2 Nov 2015 13:00:59 -0800 Subject: [PATCH 13/31] Fix failing test in UnsafeShuffleWriterSuite. --- .../org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 4763395d7d401..0e0eca515afc1 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -423,7 +423,7 @@ public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exce memoryManager.limit(UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE * 16); final UnsafeShuffleWriter writer = createWriter(false); final ArrayList> dataToWrite = new ArrayList<>(); - for (int i = 0; i < UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE; i++) { + for (int i = 0; i < UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE + 1; i++) { dataToWrite.add(new Tuple2(i, i)); } writer.write(dataToWrite.iterator()); From 8e12eb4e96b7e30a4a0a778f54c858c8597df68c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 2 Nov 2015 14:32:53 -0800 Subject: [PATCH 14/31] Add more comments. --- .../spark/memory/ExecutionMemoryPool.scala | 10 ++- .../apache/spark/memory/MemoryManager.scala | 2 +- .../org/apache/spark/memory/MemoryPool.scala | 45 +++++++++-- .../spark/memory/StorageMemoryPool.scala | 42 +++++++---- .../spark/memory/UnifiedMemoryManager.scala | 47 +++++++----- .../org/apache/spark/memory/package.scala | 75 +++++++++++++++++++ 6 files changed, 180 insertions(+), 41 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/memory/package.scala diff --git a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala index 044d472a628b8..d11f82f300192 100644 --- a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala @@ -24,7 +24,7 @@ import scala.collection.mutable import org.apache.spark.Logging /** - * Implements policies and bookkeeping for sharing a fixed-size pool of memory between tasks. + * Implements policies and bookkeeping for sharing a adjustable-sized pool of memory between tasks. * * Tries to ensure that each task gets a reasonable share of memory, instead of some task ramping up * to a large amount first and then causing others to spill to disk repeatedly. @@ -39,7 +39,10 @@ import org.apache.spark.Logging * @param memoryManager a [[MemoryManager]] instance to synchronize on * @param poolName a human-readable name for this pool, for use in log messages */ -class ExecutionMemoryPool(memoryManager: Object, poolName: String) extends MemoryPool with Logging { +class ExecutionMemoryPool( + memoryManager: Object, + poolName: String + ) extends MemoryPool(memoryManager) with Logging { /** * Map from taskAttemptId -> memory consumption in bytes @@ -115,6 +118,9 @@ class ExecutionMemoryPool(memoryManager: Object, poolName: String) extends Memor 0L // Never reached } + /** + * Release `numBytes` of memory acquired by the given task. + */ def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = memoryManager.synchronized { val curMem = memoryForTask.getOrElse(taskAttemptId, 0L) var memoryToFree = if (curMem < numBytes) { diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 385e1b46a4b4b..579dea5428efe 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -39,7 +39,7 @@ private[spark] abstract class MemoryManager( // -- Methods related to memory allocation policies and bookkeeping ------------------------------ // TODO(josh): think through and document thread-safety contracts - protected val storageMemoryPool = new StorageMemoryPool() + protected val storageMemoryPool = new StorageMemoryPool(this) protected val onHeapExecutionMemoryPool = new ExecutionMemoryPool(this, "on-heap execution") protected val offHeapExecutionMemoryPool = new ExecutionMemoryPool(this, "off-heap execution") diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala index 6822b181d4229..320388aade94f 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala @@ -17,22 +17,55 @@ package org.apache.spark.memory -abstract class MemoryPool { +import javax.annotation.concurrent.GuardedBy +/** + * Manages bookkeeping for an adjustable-sized region of memory. This class is internal to + * the [[MemoryManager]]. See subclasses for more details. + * + * @param memoryManager a [[MemoryManager]] instance, used for synchronization. We purposely + * erase the type to `Object` to avoid programming errors, since this object + * should only be used for synchronization purposes. + */ +abstract class MemoryPool(memoryManager: Object) { + + @GuardedBy("memoryManager") private[this] var _poolSize: Long = 0 - final def poolSize: Long = _poolSize - final def memoryFree: Long = _poolSize - memoryUsed - def memoryUsed: Long + /** + * Returns the current size of the pool, in bytes. + */ + final def poolSize: Long = memoryManager.synchronized { + _poolSize + } - def incrementPoolSize(delta: Long): Unit = { + /** + * Returns the amount of free memory in the pool, in bytes. + */ + final def memoryFree: Long = memoryManager.synchronized { + _poolSize - memoryUsed + } + + /** + * Expands the pool by `delta` bytes. + */ + final def incrementPoolSize(delta: Long): Unit = memoryManager.synchronized { require(delta >= 0) _poolSize += delta } - def decrementPoolSize(delta: Long): Unit = { + /** + * Shrinks the pool by `delta` bytes. + */ + final def decrementPoolSize(delta: Long): Unit = memoryManager.synchronized { require(delta >= 0) require(delta <= _poolSize) + require(_poolSize - delta >= memoryUsed) _poolSize -= delta } + + /** + * Returns the amount of used memory in this pool (in bytes). + */ + def memoryUsed: Long } diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index d05631bc4161f..7449060a68905 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -17,17 +17,28 @@ package org.apache.spark.memory -import org.apache.spark.{TaskContext, Logging} -import org.apache.spark.storage.{MemoryStore, BlockStatus, BlockId} +import javax.annotation.concurrent.GuardedBy import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -class StorageMemoryPool extends MemoryPool with Logging { +import org.apache.spark.{TaskContext, Logging} +import org.apache.spark.storage.{MemoryStore, BlockStatus, BlockId} + +/** + * Performs bookkeeping for managing an adjustable-size pool of memory that is used for storage + * (caching). + * + * @param memoryManager a [[MemoryManager]] instance to synchronize on + */ +class StorageMemoryPool(memoryManager: Object) extends MemoryPool(memoryManager) with Logging { + @GuardedBy("memoryManager") private[this] var _memoryUsed: Long = 0L - override def memoryUsed: Long = _memoryUsed + override def memoryUsed: Long = memoryManager.synchronized { + _memoryUsed + } private var _memoryStore: MemoryStore = _ def memoryStore: MemoryStore = { @@ -53,7 +64,7 @@ class StorageMemoryPool extends MemoryPool with Logging { def acquireMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = memoryManager.synchronized { acquireMemory(blockId, numBytes, numBytes, evictedBlocks) } @@ -69,7 +80,7 @@ class StorageMemoryPool extends MemoryPool with Logging { blockId: BlockId, numBytesToAcquire: Long, numBytesToFree: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = memoryManager.synchronized { assert(numBytesToAcquire >= 0) assert(numBytesToFree >= 0) assert(memoryUsed <= poolSize) @@ -90,7 +101,7 @@ class StorageMemoryPool extends MemoryPool with Logging { enoughMemory } - def releaseMemory(size: Long): Unit = synchronized { + def releaseMemory(size: Long): Unit = memoryManager.synchronized { if (size > _memoryUsed) { logWarning(s"Attempted to release $size bytes of storage " + s"memory when we only have ${_memoryUsed} bytes") @@ -100,19 +111,24 @@ class StorageMemoryPool extends MemoryPool with Logging { } } - def releaseAllMemory(): Unit = synchronized { + def releaseAllMemory(): Unit = memoryManager.synchronized { _memoryUsed = 0 } - // TODO(josh): comment - def shrinkPoolToFreeSpace(spaceToEnsure: Long): Long = synchronized { - val spaceFreedByReleasingUnusedMemory = Math.min(spaceToEnsure, memoryFree) + /** + * Try to shrink the size of this storage memory pool by `spaceToFree` bytes. Return the number + * of bytes removed from the pool's capacity. + */ + def shrinkPoolToFreeSpace(spaceToFree: Long): Long = memoryManager.synchronized { + // First, shrink the pool by reclaiming free memory: + val spaceFreedByReleasingUnusedMemory = Math.min(spaceToFree, memoryFree) decrementPoolSize(spaceFreedByReleasingUnusedMemory) - if (spaceFreedByReleasingUnusedMemory == spaceToEnsure) { + if (spaceFreedByReleasingUnusedMemory == spaceToFree) { spaceFreedByReleasingUnusedMemory } else { + // If reclaiming free memory did not adequately shrink the pool, begin evicting blocks: val evictedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - memoryStore.ensureFreeSpace(spaceToEnsure - spaceFreedByReleasingUnusedMemory, evictedBlocks) + memoryStore.ensureFreeSpace(spaceToFree - spaceFreedByReleasingUnusedMemory, evictedBlocks) val spaceFreedByEviction = evictedBlocks.map(_._2.memSize).sum _memoryUsed -= spaceFreedByEviction decrementPoolSize(spaceFreedByEviction) diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 26be295a32bad..84cbcaf40c98c 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -41,31 +41,31 @@ import org.apache.spark.storage.{BlockStatus, BlockId} * up most of the storage space, in which case the new blocks will be evicted immediately * according to their respective storage levels. * - * @param minimumStoragePoolSize Size of the storage region, in bytes. - * This region is not statically reserved; execution can borrow from - * it if necessary. Cached blocks can be evicted only if actual - * storage memory usage exceeds this region. + * @param storageRegionSize Size of the storage region, in bytes. + * This region is not statically reserved; execution can borrow from + * it if necessary. Cached blocks can be evicted only if actual + * storage memory usage exceeds this region. */ private[spark] class UnifiedMemoryManager private[memory] ( conf: SparkConf, maxMemory: Long, - private val minimumStoragePoolSize: Long, + private val storageRegionSize: Long, numCores: Int) extends MemoryManager( conf, numCores, // TODO(josh): it is confusing how this interacts with page size calculations: - maxOnHeapExecutionMemory = maxMemory - minimumStoragePoolSize) { + maxOnHeapExecutionMemory = maxMemory - storageRegionSize) { - // At first, all memory is allocated towards execution. - // TODO(josh): in light of this policy, the name minimumStoragePoolSize is confusing. + // We always maintain the invariant that + // onHeapExecutionMemoryPool.poolSize + storageMemoryPool.poolSize == maxMemory + // At first, all memory is allocated towards execution: onHeapExecutionMemoryPool.incrementPoolSize(maxMemory) override def maxStorageMemory: Long = synchronized { maxMemory - onHeapExecutionMemoryPool.memoryUsed } - /** * Try to acquire up to `numBytes` of execution memory for the current task and return the * number of bytes obtained, or 0 if none can be allocated. @@ -83,18 +83,24 @@ private[spark] class UnifiedMemoryManager private[memory] ( assert(numBytes >= 0) memoryMode match { case MemoryMode.ON_HEAP => - val memoryBorrowedByStorage = - math.max(storageMemoryPool.memoryFree, - storageMemoryPool.poolSize - minimumStoragePoolSize) - // If there is not enough free memory AND storage has borrowed some execution memory, - // then evict as much memory borrowed by storage as needed to grant this request - if (numBytes > onHeapExecutionMemoryPool.memoryFree && memoryBorrowedByStorage > 0) { - val spaceReclaimed = storageMemoryPool.shrinkPoolToFreeSpace( - math.min(numBytes, memoryBorrowedByStorage)) - onHeapExecutionMemoryPool.incrementPoolSize(spaceReclaimed) + if (numBytes > onHeapExecutionMemoryPool.memoryFree) { + // There is not enough free memory in the execution pool, so try to reclaim memory from + // storage. We can reclaim any free memory from the storage pool. If the storage pool + // has grown to become larger than `storageRegionSize`, we can evict blocks and reclaim + // the memory that storage has borrowed from execution. + val memoryReclaimableFromStorage = + math.max(storageMemoryPool.memoryFree, storageMemoryPool.poolSize - storageRegionSize) + if (memoryReclaimableFromStorage > 0) { + // Only reclaim as much space as is necessary and available: + val spaceReclaimed = storageMemoryPool.shrinkPoolToFreeSpace( + math.min(numBytes, memoryReclaimableFromStorage)) + onHeapExecutionMemoryPool.incrementPoolSize(spaceReclaimed) + } } onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) case MemoryMode.OFF_HEAP => + // For now, we only support on-heap caching of data, so we do not need to interact with + // the storage pool when allocating off-heap memory. This will change in the future, though. super.acquireExecutionMemory(numBytes, taskAttemptId, memoryMode) } } @@ -104,7 +110,10 @@ private[spark] class UnifiedMemoryManager private[memory] ( numBytes: Long, evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { assert(onHeapExecutionMemoryPool.poolSize + storageMemoryPool.poolSize == maxMemory) + assert(numBytes >= 0) if (numBytes > storageMemoryPool.memoryFree) { + // There is not enough free memory in the storage pool, so try to borrow free memory from + // the execution pool. val memoryBorrowedFromExecution = Math.min(onHeapExecutionMemoryPool.memoryFree, numBytes) onHeapExecutionMemoryPool.decrementPoolSize(memoryBorrowedFromExecution) storageMemoryPool.incrementPoolSize(memoryBorrowedFromExecution) @@ -127,7 +136,7 @@ object UnifiedMemoryManager { new UnifiedMemoryManager( conf, maxMemory = maxMemory, - minimumStoragePoolSize = + storageRegionSize = (maxMemory * conf.getDouble("spark.memory.storageFraction", 0.5)).toLong, numCores = numCores) } diff --git a/core/src/main/scala/org/apache/spark/memory/package.scala b/core/src/main/scala/org/apache/spark/memory/package.scala new file mode 100644 index 0000000000000..4fcbdec4aec5d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/memory/package.scala @@ -0,0 +1,75 @@ +/* + * 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 + +/** + * This package implements Spark's memory management system. This system consists of two main + * components, a JVM-wide memory manager and a per-task manager: + * + * - [[org.apache.spark.memory.MemoryManager]] manages Spark's overall memory usage within a JVM. + * This component implements the policies for dividing the available memory across tasks and for + * allocating memory between storage (memory used caching and data transfer) and execution (memory + * used by computations, such as shuffles, joins, sorts, and aggregations). + * - [[org.apache.spark.memory.TaskMemoryManager]] manages the memory allocated by individual tasks. + * Tasks interact with TaskMemoryManager and never directly interact with the JVM-wide + * MemoryManager. + * + * Internally, each of these components have additional abstractions for memory bookkeeping: + * + * - [[org.apache.spark.memory.MemoryConsumer]]s are clients of the TaskMemoryManager and + * correspond to individual operators and data structures within a task. The TaskMemoryManager + * receives memory allocation requests from MemoryConsumers and issues callbacks to consumers + * in order to trigger spilling when running low on memory. + * - [[org.apache.spark.memory.MemoryPool]]s are a bookkeeping abstraction used by the + * MemoryManager to track the division of memory between storage and execution. + * + * Diagrammatically: + * + * {{{ + * +-------------+ + * | MemConsumer |----+ +------------------------+ + * +-------------+ | +-------------------+ | MemoryManager | + * +--->| TaskMemoryManager |----+ | | + * +-------------+ | +-------------------+ | | +------------------+ | + * | MemConsumer |----+ | | | StorageMemPool | | + * +-------------+ +-------------------+ | | +------------------+ | + * | TaskMemoryManager |----+ | | + * +-------------------+ | | +------------------+ | + * +---->| |OnHeapExecMemPool | | + * * | | +------------------+ | + * * | | | + * +-------------+ * | | +------------------+ | + * | MemConsumer |----+ | | |OffHeapExecMemPool| | + * +-------------+ | +-------------------+ | | +------------------+ | + * +--->| TaskMemoryManager |----+ | | + * +-------------------+ +------------------------+ + * }}} + * + * + * There are two implementations of [[org.apache.spark.memory.MemoryManager]] which vary in how + * they handle the sizing of their memory pools: + * + * - [[org.apache.spark.memory.UnifiedMemoryManager]], the default in Spark 1.6+, enforces soft + * boundaries between storage and execution memory, allowing requests for memory in one region + * to be fulfilled by borrowing memory from the other. + * - [[org.apache.spark.memory.StaticMemoryManager]] enforces hard boundaries between storage + * and execution memory by statically partitioning Spark's memory and preventing storage and + * execution from borrowing memory from each other. This mode is retained only for legacy + * compatibility purposes. + */ +package object memory \ No newline at end of file From 418f9b3c1a6dd56076f4090c7c5b503a055e76a0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 2 Nov 2015 14:43:58 -0800 Subject: [PATCH 15/31] Fix TODO related to logging of non-consumer memory. --- .../org/apache/spark/memory/TaskMemoryManager.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 130e941e63003..fba14c4211432 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -202,17 +202,23 @@ public void releaseExecutionMemory(long size, MemoryMode mode, MemoryConsumer co /** * Dump the memory usage of all consumers. */ - // TODO(josh): also report memory not allocated to any particular consumer. public void showMemoryUsage() { logger.info("Memory used in task " + taskAttemptId); synchronized (this) { + long memoryAccountedForByConsumers = 0; for (MemoryConsumer c: consumers) { long totalMemUsage = c.getMemoryUsed(MemoryMode.OFF_HEAP) + c.getMemoryUsed(MemoryMode.ON_HEAP); + memoryAccountedForByConsumers += totalMemUsage; if (totalMemUsage > 0) { logger.info("Acquired by " + c + ": " + Utils.bytesToString(totalMemUsage)); } } + long memoryNotAccountedFor = + memoryManager.getExecutionMemoryUsageForTask(taskAttemptId) - memoryAccountedForByConsumers; + logger.info( + "{} bytes of memory were used by task {} but are not associated with specific consumers", + memoryNotAccountedFor, taskAttemptId); } } @@ -376,7 +382,7 @@ public long cleanUpAllAllocatedMemory() { } /** - * Returns the memory consumption, in bytes, for the current task + * Returns the memory consumption, in bytes, for the current task. */ public long getMemoryConsumptionForThisTask() { return memoryManager.getExecutionMemoryUsageForTask(taskAttemptId); From b0d569d4cce9eb7515fa994c320579b107aa2f17 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 2 Nov 2015 14:44:44 -0800 Subject: [PATCH 16/31] Add missing newline for Scalastyle. --- core/src/main/scala/org/apache/spark/memory/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/memory/package.scala b/core/src/main/scala/org/apache/spark/memory/package.scala index 4fcbdec4aec5d..564e30d2ffd66 100644 --- a/core/src/main/scala/org/apache/spark/memory/package.scala +++ b/core/src/main/scala/org/apache/spark/memory/package.scala @@ -72,4 +72,4 @@ package org.apache.spark * execution from borrowing memory from each other. This mode is retained only for legacy * compatibility purposes. */ -package object memory \ No newline at end of file +package object memory From 820fa380721dd3f66f401c34d3f7cb303b12f819 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 2 Nov 2015 14:56:22 -0800 Subject: [PATCH 17/31] Document thread-safety. --- .../scala/org/apache/spark/memory/MemoryManager.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 579dea5428efe..ebed719ab663c 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -17,6 +17,8 @@ package org.apache.spark.memory +import javax.annotation.concurrent.GuardedBy + import scala.collection.mutable import org.apache.spark.{SparkConf, Logging} @@ -38,9 +40,11 @@ private[spark] abstract class MemoryManager( // -- Methods related to memory allocation policies and bookkeeping ------------------------------ - // TODO(josh): think through and document thread-safety contracts + @GuardedBy("this") protected val storageMemoryPool = new StorageMemoryPool(this) + @GuardedBy("this") protected val onHeapExecutionMemoryPool = new ExecutionMemoryPool(this, "on-heap execution") + @GuardedBy("this") protected val offHeapExecutionMemoryPool = new ExecutionMemoryPool(this, "off-heap execution") offHeapExecutionMemoryPool.incrementPoolSize(conf.getSizeAsBytes("spark.memory.offHeapSize", 0)) @@ -56,7 +60,7 @@ private[spark] abstract class MemoryManager( * Set the [[MemoryStore]] used by this manager to evict cached blocks. * This must be set after construction due to initialization ordering constraints. */ - final def setMemoryStore(store: MemoryStore): Unit = { + final def setMemoryStore(store: MemoryStore): Unit = synchronized { storageMemoryPool.setMemoryStore(store) } @@ -70,7 +74,7 @@ private[spark] abstract class MemoryManager( def acquireStorageMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = { + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = synchronized { storageMemoryPool.acquireMemory(blockId, numBytes, evictedBlocks) } From f976faa6461ac4d74702c970350f4103663fe3c8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 2 Nov 2015 14:58:09 -0800 Subject: [PATCH 18/31] Update comment about spilling + tungsten pages. --- .../main/java/org/apache/spark/memory/MemoryConsumer.java | 3 ++- .../java/org/apache/spark/memory/TaskMemoryManager.java | 7 ++----- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 94a9d6a91c61a..b5ed7662c5b6c 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -68,12 +68,13 @@ public void spill() throws IOException { * * Note: In order to avoid possible deadlock, should not call acquireMemory() from spill(). * + * Note: today, this only frees Tungsten-managed pages. + * * @param size the amount of memory should be released * @param trigger the MemoryConsumer that trigger this spilling * @return the amount of released memory in bytes * @throws IOException */ - // TODO(josh): clarify assumption that this only frees Tungsten-managed pages (for now). public abstract long spill(long size, MemoryConsumer trigger) throws IOException; /** diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index fba14c4211432..01007a574a844 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -132,23 +132,20 @@ public long acquireExecutionMemory( MemoryMode mode, MemoryConsumer consumer) { assert(required >= 0); - // TODO(josh): handle spill differently based on type of request (on-heap vs off-heap). - // If we are allocating tungsten pages off-heap and receive a request to allocate on-heap + // If we are allocating Tungsten pages off-heap and receive a request to allocate on-heap // memory here, then it may not make sense to spill since that would only end up freeing // off-heap memory. This is subject to change, though, so it may be risky to make this // optimization now in case we forget to undo it late when making changes. synchronized (this) { long got = memoryManager.acquireExecutionMemory(required, taskAttemptId, mode); - // try to release memory from other consumers first, then we can reduce the frequency of + // Try to release memory from other consumers first, then we can reduce the frequency of // spilling, avoid to have too many spilled files. if (got < required) { // Call spill() on other consumers to release memory for (MemoryConsumer c: consumers) { if (c != consumer && c.getMemoryUsed(mode) > 0) { try { - // TODO(josh): subtlety / implementation detail: today, spill() happens to only - // release Tungsten pages. long released = c.spill(required - got, consumer); if (released > 0 && mode == tungstenMemoryMode) { logger.info("Task {} released {} from {} for {}", taskAttemptId, From 71cd9b015741228e2ec70cb5621ceb750befaff9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 2 Nov 2015 15:12:38 -0800 Subject: [PATCH 19/31] Import ordering --- .../test/scala/org/apache/spark/memory/MemoryManagerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 544c50dbe408a..84d3adf6c9487 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -30,7 +30,7 @@ import org.mockito.stubbing.Answer import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkFunSuite -import org.apache.spark.storage.{StorageLevel, BlockStatus, BlockId, MemoryStore} +import org.apache.spark.storage.{BlockId, BlockStatus, MemoryStore, StorageLevel} /** From 852b5c9b4641fb8be098ff505444d62417fa3b80 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 2 Nov 2015 15:29:15 -0800 Subject: [PATCH 20/31] Address commented-out test asserts in UnifiedMemoryManagerSuite. --- .../memory/UnifiedMemoryManagerSuite.scala | 41 +------------------ 1 file changed, 1 insertion(+), 40 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 8c15d322f8ce8..c89771471cb3f 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -47,18 +47,6 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes UnifiedMemoryManager(conf, numCores = 1) } -// private def getStorageRegionSize(mm: UnifiedMemoryManager): Long = { -// (mm invokePrivate PrivateMethod[StorageMemoryPool]('storageMemoryPool)()).poolSize -// } - -// test("storage region size") { -// val maxMemory = 1000L -// val (mm, _) = makeThings(maxMemory) -// val expectedStorageRegionSize = maxMemory * storageFraction -// val actualStorageRegionSize = getStorageRegionSize(mm) -// assert(expectedStorageRegionSize === actualStorageRegionSize) -// } - test("basic execution memory") { val maxMemory = 1000L val taskAttemptId = 0L @@ -124,22 +112,11 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes val maxMemory = 1000L val taskAttemptId = 0L val (mm, ms) = makeThings(maxMemory) - // First, ensure the test classes are set up as expected - val expectedStorageRegionSize = 500L - val expectedExecutionRegionSize = 500L -// val storageRegionSize = getStorageRegionSize(mm) - val executionRegionSize = maxMemory - expectedStorageRegionSize -// require(storageRegionSize === expectedStorageRegionSize, -// "bad test: storage region size is unexpected") - require(executionRegionSize === expectedExecutionRegionSize, - "bad test: storage region size is unexpected") // Acquire enough storage memory to exceed the storage region assert(mm.acquireStorageMemory(dummyBlock, 750L, evictedBlocks)) assertEnsureFreeSpaceCalled(ms, 750L) assert(mm.executionMemoryUsed === 0L) assert(mm.storageMemoryUsed === 750L) -// require(mm.storageMemoryUsed > storageRegionSize, -// s"bad test: storage memory used should exceed the storage region") // Execution needs to request 250 bytes to evict storage memory assert(mm.acquireExecutionMemory(100L, taskAttemptId, MemoryMode.ON_HEAP) === 100L) assert(mm.executionMemoryUsed === 100L) @@ -150,16 +127,13 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes assertEnsureFreeSpaceCalled(ms, 200L) assert(mm.executionMemoryUsed === 300L) mm.releaseAllStorageMemory() - require(mm.executionMemoryUsed < executionRegionSize, - s"bad test: execution memory used should be within the execution region") + require(mm.executionMemoryUsed === 300L) require(mm.storageMemoryUsed === 0, "bad test: all storage memory should have been released") // Acquire some storage memory again, but this time keep it within the storage region assert(mm.acquireStorageMemory(dummyBlock, 400L, evictedBlocks)) assertEnsureFreeSpaceCalled(ms, 400L) assert(mm.storageMemoryUsed === 400L) assert(mm.executionMemoryUsed === 300L) -// require(mm.storageMemoryUsed < storageRegionSize, -// s"bad test: storage memory used should be within the storage region") // Execution cannot evict storage because the latter is within the storage fraction, // so grant only what's remaining without evicting anything, i.e. 1000 - 300 - 400 = 300 assert(mm.acquireExecutionMemory(400L, taskAttemptId, MemoryMode.ON_HEAP) === 300L) @@ -172,22 +146,11 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes val maxMemory = 1000L val taskAttemptId = 0L val (mm, ms) = makeThings(maxMemory) - // First, ensure the test classes are set up as expected - val expectedStorageRegionSize = 500L - val expectedExecutionRegionSize = 500L -// val storageRegionSize = getStorageRegionSize(mm) - val executionRegionSize = maxMemory - expectedStorageRegionSize -// require(storageRegionSize === expectedStorageRegionSize, -// "bad test: storage region size is unexpected") - require(executionRegionSize === expectedExecutionRegionSize, - "bad test: storage region size is unexpected") // Acquire enough execution memory to exceed the execution region assert(mm.acquireExecutionMemory(800L, taskAttemptId, MemoryMode.ON_HEAP) === 800L) assert(mm.executionMemoryUsed === 800L) assert(mm.storageMemoryUsed === 0L) assertEnsureFreeSpaceNotCalled(ms) - require(mm.executionMemoryUsed > executionRegionSize, - s"bad test: execution memory used should exceed the execution region") // Storage should not be able to evict execution assert(mm.acquireStorageMemory(dummyBlock, 100L, evictedBlocks)) assert(mm.executionMemoryUsed === 800L) @@ -204,8 +167,6 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes assert(mm.executionMemoryUsed === 200L) assert(mm.storageMemoryUsed === 0L) assertEnsureFreeSpaceNotCalled(ms) - require(mm.executionMemoryUsed < executionRegionSize, - s"bad test: execution memory used should be within the execution region") // Storage should still not be able to evict execution assert(mm.acquireStorageMemory(dummyBlock, 750L, evictedBlocks)) assert(mm.executionMemoryUsed === 200L) From 6a5204f8de5eb86d5901374bd7aa4d678fba348a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 2 Nov 2015 15:42:59 -0800 Subject: [PATCH 21/31] Add rudimentary test for off-heap accounting. --- .../spark/memory/MemoryManagerSuite.scala | 34 +++++++++++++++---- .../memory/StaticMemoryManagerSuite.scala | 11 ++++-- .../memory/UnifiedMemoryManagerSuite.scala | 7 ++-- 3 files changed, 41 insertions(+), 11 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 84d3adf6c9487..7bc19773047ea 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -138,16 +138,18 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { } /** - * Create a MemoryManager with the specified execution memory limit and no storage memory. + * Create a MemoryManager with the specified execution memory limits and no storage memory. */ - protected def createMemoryManager(maxExecutionMemory: Long): MemoryManager + protected def createMemoryManager( + maxOnHeapExecutionMemory: Long, + maxOffHeapExecutionMemory: Long = 0L): MemoryManager // -- Tests of sharing of execution memory between tasks ---------------------------------------- // Prior to Spark 1.6, these tests were part of ShuffleMemoryManagerSuite. implicit val ec = ExecutionContext.global - test("single task requesting execution memory") { + test("single task requesting on-heap execution memory") { val manager = createMemoryManager(1000L) val taskMemoryManager = new TaskMemoryManager(manager, 0) @@ -167,7 +169,7 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { assert(taskMemoryManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) === 0L) } - test("two tasks requesting full execution memory") { + test("two tasks requesting full on-heap execution memory") { val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) @@ -187,7 +189,7 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { assert(Await.result(t2Result2, 200.millis) === 0L) } - test("two tasks cannot grow past 1 / N of execution memory") { + test("two tasks cannot grow past 1 / N of on-heap execution memory") { val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) @@ -207,7 +209,7 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { assert(Await.result(t2Result2, futureTimeout) === 250L) } - test("tasks can block to get at least 1 / 2N of execution memory") { + test("tasks can block to get at least 1 / 2N of on-heap execution memory") { val memoryManager = createMemoryManager(1000L) val t1MemManager = new TaskMemoryManager(memoryManager, 1) val t2MemManager = new TaskMemoryManager(memoryManager, 2) @@ -266,6 +268,26 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) } assert(Await.result(t1Result2, 200.millis) === 0L) } + + test("off-heap execution allocations cannot exceed limit") { + val memoryManager = createMemoryManager( + maxOnHeapExecutionMemory = 0L, + maxOffHeapExecutionMemory = 1000L) + + val tMemManager = new TaskMemoryManager(memoryManager, 1) + val result1 = Future { tMemManager.acquireExecutionMemory(1000L, MemoryMode.OFF_HEAP, null) } + assert(Await.result(result1, 200.millis) === 1000L) + assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) + + val result2 = Future { tMemManager.acquireExecutionMemory(300L, MemoryMode.OFF_HEAP, null) } + assert(Await.result(result2, 200.millis) === 0L) + + assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) + tMemManager.releaseExecutionMemory(500L, MemoryMode.OFF_HEAP, null) + assert(tMemManager.getMemoryConsumptionForThisTask === 500L) + tMemManager.releaseExecutionMemory(500L, MemoryMode.OFF_HEAP, null) + assert(tMemManager.getMemoryConsumptionForThisTask === 0l) + } } private object MemoryManagerSuite { diff --git a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala index d442bf4c8462c..54cb28c389c2f 100644 --- a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala @@ -43,10 +43,15 @@ class StaticMemoryManagerSuite extends MemoryManagerSuite { (mm, ms) } - override protected def createMemoryManager(maxMemory: Long): MemoryManager = { + override protected def createMemoryManager( + maxOnHeapExecutionMemory: Long, + maxOffHeapExecutionMemory: Long): StaticMemoryManager = { new StaticMemoryManager( - conf, - maxOnHeapExecutionMemory = maxMemory, + conf.clone + .set("spark.memory.fraction", "1") + .set("spark.testing.memory", maxOnHeapExecutionMemory.toString) + .set("spark.memory.offHeapSize", maxOffHeapExecutionMemory.toString), + maxOnHeapExecutionMemory = maxOnHeapExecutionMemory, maxStorageMemory = 0, numCores = 1) } diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index c89771471cb3f..3fb387ca92b98 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -39,10 +39,13 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes (mm, ms) } - override protected def createMemoryManager(maxMemory: Long): UnifiedMemoryManager = { + override protected def createMemoryManager( + maxOnHeapExecutionMemory: Long, + maxOffHeapExecutionMemory: Long): UnifiedMemoryManager = { val conf = new SparkConf() .set("spark.memory.fraction", "1") - .set("spark.testing.memory", maxMemory.toString) + .set("spark.testing.memory", maxOnHeapExecutionMemory.toString) + .set("spark.memory.offHeapSize", maxOffHeapExecutionMemory.toString) .set("spark.memory.storageFraction", storageFraction.toString) UnifiedMemoryManager(conf, numCores = 1) } From 9ec29c31c9c9c6a1d1676585659cbc8234d04160 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 3 Nov 2015 00:30:46 -0800 Subject: [PATCH 22/31] Fix scalastyle. --- .../test/scala/org/apache/spark/memory/MemoryManagerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 7bc19773047ea..f55d435fa33a6 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -286,7 +286,7 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite { tMemManager.releaseExecutionMemory(500L, MemoryMode.OFF_HEAP, null) assert(tMemManager.getMemoryConsumptionForThisTask === 500L) tMemManager.releaseExecutionMemory(500L, MemoryMode.OFF_HEAP, null) - assert(tMemManager.getMemoryConsumptionForThisTask === 0l) + assert(tMemManager.getMemoryConsumptionForThisTask === 0L) } } From 2fb52b59f9cb855ea364627394bc1fa4522dde28 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 3 Nov 2015 00:33:44 -0800 Subject: [PATCH 23/31] De-allocate pages when freeing leaked memory in TaskMemoryManager. --- .../java/org/apache/spark/memory/TaskMemoryManager.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 01007a574a844..4e484b0c3d459 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -359,7 +359,6 @@ public long getOffsetInPage(long pagePlusOffsetAddress) { */ public long cleanUpAllAllocatedMemory() { synchronized (this) { - Arrays.fill(pageTable, null); for (MemoryConsumer c: consumers) { if (c.getMemoryUsed(MemoryMode.ON_HEAP) > 0) { // In case of failed task, it's normal to see leaked memory @@ -375,6 +374,13 @@ public long cleanUpAllAllocatedMemory() { consumers.clear(); } + for (MemoryBlock page : pageTable) { + if (page != null) { + memoryManager.tungstenMemoryAllocator().free(page); + } + } + Arrays.fill(pageTable, null); + return memoryManager.releaseAllExecutionMemoryForTask(taskAttemptId); } From 96705b881caa82b87874d5145c16ce232c4a64a4 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 3 Nov 2015 00:35:16 -0800 Subject: [PATCH 24/31] Comment rewording --- core/src/main/java/org/apache/spark/memory/MemoryConsumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index b5ed7662c5b6c..a9dee44832347 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -41,7 +41,7 @@ protected MemoryConsumer(TaskMemoryManager taskMemoryManager) { } /** - * Returns the size of used on-heap memory in bytes. + * Returns the size, in bytes, of used memory of the specified mode. */ long getMemoryUsed(MemoryMode mode) { if (mode == MemoryMode.ON_HEAP) { From b5fb705ae7565ef0601c8d8cf2b426b7e1529674 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 3 Nov 2015 16:45:25 -0800 Subject: [PATCH 25/31] Rename acquireMemory to acquireOnHeapMemory in order to disambiguate. --- .../org/apache/spark/memory/MemoryConsumer.java | 6 +++--- .../spark/shuffle/sort/ShuffleExternalSorter.java | 14 +++++++------- .../apache/spark/unsafe/map/BytesToBytesMap.java | 6 +++--- .../unsafe/sort/UnsafeExternalSorter.java | 14 +++++++------- .../spark/memory/TaskMemoryManagerSuite.java | 6 +++--- 5 files changed, 23 insertions(+), 23 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index a9dee44832347..3dc9c9741d56d 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -82,7 +82,7 @@ public void spill() throws IOException { * * If there is not enough memory, throws OutOfMemoryError. */ - protected void acquireMemory(long size) { + protected void acquireOnHeapMemory(long size) { long got = taskMemoryManager.acquireExecutionMemory(size, MemoryMode.ON_HEAP, this); if (got < size) { taskMemoryManager.releaseExecutionMemory(got, MemoryMode.ON_HEAP, this); @@ -93,9 +93,9 @@ protected void acquireMemory(long size) { } /** - * Release `size` bytes memory. + * Release `size` bytes of on-heap memory. */ - protected void releaseMemory(long size) { + protected void releaseOnHeapMemory(long size) { onHeapMemoryUsed -= size; taskMemoryManager.releaseExecutionMemory(size, MemoryMode.ON_HEAP, this); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 400d8520019b9..f8f3444dbd748 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -114,7 +114,7 @@ public ShuffleExternalSorter( this.numElementsForSpillThreshold = conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", Long.MAX_VALUE); this.writeMetrics = writeMetrics; - acquireMemory(initialSize * 8L); + acquireOnHeapMemory(initialSize * 8L); this.inMemSorter = new ShuffleInMemorySorter(initialSize); this.peakMemoryUsedBytes = getMemoryUsage(); } @@ -303,7 +303,7 @@ public void cleanupResources() { if (inMemSorter != null) { long sorterMemoryUsage = inMemSorter.getMemoryUsage(); inMemSorter = null; - releaseMemory(sorterMemoryUsage); + releaseOnHeapMemory(sorterMemoryUsage); } for (SpillInfo spill : spills) { if (spill.file.exists() && !spill.file.delete()) { @@ -323,7 +323,7 @@ private void growPointerArrayIfNecessary() throws IOException { long used = inMemSorter.getMemoryUsage(); long needed = used + inMemSorter.getMemoryToExpand(); try { - acquireMemory(needed); // could trigger spilling + acquireOnHeapMemory(needed); // could trigger spilling } catch (OutOfMemoryError e) { // should have trigger spilling assert(inMemSorter.hasSpaceForAnotherRecord()); @@ -331,14 +331,14 @@ private void growPointerArrayIfNecessary() throws IOException { } // check if spilling is triggered or not if (inMemSorter.hasSpaceForAnotherRecord()) { - releaseMemory(needed); + releaseOnHeapMemory(needed); } else { try { inMemSorter.expandPointerArray(); - releaseMemory(used); + releaseOnHeapMemory(used); } catch (OutOfMemoryError oom) { // Just in case that JVM had run out of memory - releaseMemory(needed); + releaseOnHeapMemory(needed); spill(); } } @@ -406,7 +406,7 @@ public SpillInfo[] closeAndGetSpills() throws IOException { freeMemory(); long sorterMemoryUsage = inMemSorter.getMemoryUsage(); inMemSorter = null; - releaseMemory(sorterMemoryUsage); + releaseOnHeapMemory(sorterMemoryUsage); } return spills.toArray(new SpillInfo[spills.size()]); } catch (IOException e) { diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index e36709c6fc849..94fc2ab7e9183 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -732,7 +732,7 @@ private void allocate(int capacity) { // The capacity needs to be divisible by 64 so that our bit set can be sized properly capacity = Math.max((int) Math.min(MAX_CAPACITY, ByteArrayMethods.nextPowerOf2(capacity)), 64); assert (capacity <= MAX_CAPACITY); - acquireMemory(capacity * 16); + acquireOnHeapMemory(capacity * 16); longArray = new LongArray(MemoryBlock.fromLongArray(new long[capacity * 2])); bitset = new BitSet(MemoryBlock.fromLongArray(new long[capacity / 64])); @@ -748,7 +748,7 @@ public void freeArray() { if (longArray != null) { long used = longArray.memoryBlock().size(); longArray = null; - releaseMemory(used); + releaseOnHeapMemory(used); bitset = null; } } @@ -895,7 +895,7 @@ void growAndRehash() { } } } - releaseMemory(oldLongArray.memoryBlock().size()); + releaseOnHeapMemory(oldLongArray.memoryBlock().size()); if (enablePerfMetrics) { timeSpentResizingNs += System.nanoTime() - resizeStartTime; diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 49a5a4b13b70d..8eba676f87920 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -121,7 +121,7 @@ private UnsafeExternalSorter( if (existingInMemorySorter == null) { this.inMemSorter = new UnsafeInMemorySorter(taskMemoryManager, recordComparator, prefixComparator, initialSize); - acquireMemory(inMemSorter.getMemoryUsage()); + acquireOnHeapMemory(inMemSorter.getMemoryUsage()); } else { this.inMemSorter = existingInMemorySorter; // will acquire after free the map @@ -276,7 +276,7 @@ public void cleanupResources() { if (inMemSorter != null) { long used = inMemSorter.getMemoryUsage(); inMemSorter = null; - releaseMemory(used); + releaseOnHeapMemory(used); } } } @@ -292,7 +292,7 @@ private void growPointerArrayIfNecessary() throws IOException { long used = inMemSorter.getMemoryUsage(); long needed = used + inMemSorter.getMemoryToExpand(); try { - acquireMemory(needed); // could trigger spilling + acquireOnHeapMemory(needed); // could trigger spilling } catch (OutOfMemoryError e) { // should have trigger spilling assert(inMemSorter.hasSpaceForAnotherRecord()); @@ -300,14 +300,14 @@ private void growPointerArrayIfNecessary() throws IOException { } // check if spilling is triggered or not if (inMemSorter.hasSpaceForAnotherRecord()) { - releaseMemory(needed); + releaseOnHeapMemory(needed); } else { try { inMemSorter.expandPointerArray(); - releaseMemory(used); + releaseOnHeapMemory(used); } catch (OutOfMemoryError oom) { // Just in case that JVM had run out of memory - releaseMemory(needed); + releaseOnHeapMemory(needed); spill(); } } @@ -484,7 +484,7 @@ public void loadNext() throws IOException { assert(inMemSorter != null); long used = inMemSorter.getMemoryUsage(); inMemSorter = null; - releaseMemory(used); + releaseOnHeapMemory(used); } numRecords--; upstream.loadNext(); diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index c1e0f84bce2f5..63af866628374 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -35,16 +35,16 @@ class TestMemoryConsumer extends MemoryConsumer { @Override public long spill(long size, MemoryConsumer trigger) throws IOException { long used = getMemoryUsed(taskMemoryManager.tungstenMemoryMode); - releaseMemory(used); + releaseOnHeapMemory(used); return used; } void use(long size) { - acquireMemory(size); + acquireOnHeapMemory(size); } void free(long size) { - releaseMemory(size); + releaseOnHeapMemory(size); } } From a0c5668fa3044d3934cd3a5934794a5ae569838b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 3 Nov 2015 16:47:29 -0800 Subject: [PATCH 26/31] Change exception and log levels --- .../main/java/org/apache/spark/memory/TaskMemoryManager.java | 4 ++-- .../scala/org/apache/spark/memory/StorageMemoryPool.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 4e484b0c3d459..9c68f283648f1 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -148,7 +148,7 @@ public long acquireExecutionMemory( try { long released = c.spill(required - got, consumer); if (released > 0 && mode == tungstenMemoryMode) { - logger.info("Task {} released {} from {} for {}", taskAttemptId, + logger.debug("Task {} released {} from {} for {}", taskAttemptId, Utils.bytesToString(released), c, consumer); got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); if (got >= required) { @@ -169,7 +169,7 @@ public long acquireExecutionMemory( try { long released = consumer.spill(required - got, consumer); if (released > 0 && mode == tungstenMemoryMode) { - logger.info("Task {} released {} from itself ({})", taskAttemptId, + logger.debug("Task {} released {} from itself ({})", taskAttemptId, Utils.bytesToString(released), consumer); got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); } diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index 7449060a68905..c217f019957d5 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -43,7 +43,7 @@ class StorageMemoryPool(memoryManager: Object) extends MemoryPool(memoryManager) private var _memoryStore: MemoryStore = _ def memoryStore: MemoryStore = { if (_memoryStore == null) { - throw new IllegalArgumentException("memory store not initialized yet") + throw new IllegalStateException("memory store not initialized yet") } _memoryStore } From c761736643c1053ae04aed4e1ab227709a0ad418 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 6 Nov 2015 02:45:24 -0800 Subject: [PATCH 27/31] Fix test compilation. --- .../spark/memory/TaskMemoryManagerSuite.java | 32 +++++++++---------- .../spark/memory/TestMemoryConsumer.java | 10 ++++-- 2 files changed, 24 insertions(+), 18 deletions(-) diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index b323787c5d3b3..711eed0193bc0 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -71,33 +71,33 @@ public void cooperativeSpilling() { TestMemoryConsumer c1 = new TestMemoryConsumer(manager); TestMemoryConsumer c2 = new TestMemoryConsumer(manager); c1.use(100); - assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 100); + assert(c1.getUsed() == 100); c2.use(100); - assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 100); - assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 0); // spilled + assert(c2.getUsed() == 100); + assert(c1.getUsed() == 0); // spilled c1.use(100); - assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 100); - assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 0); // spilled + assert(c1.getUsed() == 100); + assert(c2.getUsed() == 0); // spilled c1.use(50); - assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 50); // spilled - assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 0); + assert(c1.getUsed() == 50); // spilled + assert(c2.getUsed() == 0); c2.use(50); - assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 50); - assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 50); + assert(c1.getUsed() == 50); + assert(c2.getUsed() == 50); c1.use(100); - assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 100); - assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 0); // spilled + assert(c1.getUsed() == 100); + assert(c2.getUsed() == 0); // spilled c1.free(20); - assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 80); + assert(c1.getUsed() == 80); c2.use(10); - assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 80); - assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 10); + assert(c1.getUsed() == 80); + assert(c2.getUsed() == 10); c2.use(100); - assert(c2.getMemoryUsed(MemoryMode.ON_HEAP) == 100); - assert(c1.getMemoryUsed(MemoryMode.ON_HEAP) == 0); // spilled + assert(c2.getUsed() == 100); + assert(c1.getUsed() == 0); // spilled c1.free(0); c2.free(100); diff --git a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java index 8ae3642738509..e6e16fff80401 100644 --- a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java +++ b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java @@ -32,13 +32,19 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { } void use(long size) { - long got = taskMemoryManager.acquireExecutionMemory(size, this); + long got = taskMemoryManager.acquireExecutionMemory( + size, + taskMemoryManager.tungstenMemoryMode, + this); used += got; } void free(long size) { used -= size; - taskMemoryManager.releaseExecutionMemory(size, this); + taskMemoryManager.releaseExecutionMemory( + size, + taskMemoryManager.tungstenMemoryMode, + this); } } From 32398bb3f263534a1e833677bb9237c5b69e94db Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 6 Nov 2015 14:05:44 -0800 Subject: [PATCH 28/31] Address review feedback and fix bug. --- .../spark/memory/TaskMemoryManager.java | 2 +- .../spark/memory/ExecutionMemoryPool.scala | 24 +++++++++--------- .../apache/spark/memory/MemoryManager.scala | 25 ++++++++++++------- .../org/apache/spark/memory/MemoryPool.scala | 18 ++++++------- .../spark/memory/StaticMemoryManager.scala | 9 ++++--- .../spark/memory/StorageMemoryPool.scala | 18 ++++++------- .../spark/memory/UnifiedMemoryManager.scala | 12 ++++----- .../spark/memory/TestMemoryManager.scala | 10 +++----- .../memory/UnifiedMemoryManagerSuite.scala | 3 ++- .../sql/execution/UnsafeKVExternalSorter.java | 1 - 10 files changed, 63 insertions(+), 59 deletions(-) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index d8a7968668fb4..5f743b28857b4 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -253,7 +253,7 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { } allocatedPages.set(pageNumber); } - final MemoryBlock page = memoryManager.tungstenMemoryAllocator().allocate(size); + final MemoryBlock page = memoryManager.tungstenMemoryAllocator().allocate(acquired); page.pageNumber = pageNumber; pageTable[pageNumber] = page; if (logger.isTraceEnabled()) { diff --git a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala index d11f82f300192..7825bae425877 100644 --- a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala @@ -36,28 +36,28 @@ import org.apache.spark.Logging * notifyAll() to signal changes to callers. Prior to Spark 1.6, this arbitration of memory across * tasks was performed by the ShuffleMemoryManager. * - * @param memoryManager a [[MemoryManager]] instance to synchronize on + * @param lock a [[MemoryManager]] instance to synchronize on * @param poolName a human-readable name for this pool, for use in log messages */ class ExecutionMemoryPool( - memoryManager: Object, + lock: Object, poolName: String - ) extends MemoryPool(memoryManager) with Logging { + ) extends MemoryPool(lock) with Logging { /** * Map from taskAttemptId -> memory consumption in bytes */ - @GuardedBy("memoryManager") + @GuardedBy("lock") private val memoryForTask = new mutable.HashMap[Long, Long]() - override def memoryUsed: Long = memoryManager.synchronized { + override def memoryUsed: Long = lock.synchronized { memoryForTask.values.sum } /** * Returns the memory consumption, in bytes, for the given task. */ - def getMemoryUsageForTask(taskAttemptId: Long): Long = memoryManager.synchronized { + def getMemoryUsageForTask(taskAttemptId: Long): Long = lock.synchronized { memoryForTask.getOrElse(taskAttemptId, 0L) } @@ -72,7 +72,7 @@ class ExecutionMemoryPool( * * @return the number of bytes granted to the task. */ - def acquireMemory(numBytes: Long, taskAttemptId: Long): Long = memoryManager.synchronized { + def acquireMemory(numBytes: Long, taskAttemptId: Long): Long = lock.synchronized { assert(numBytes > 0, s"invalid number of bytes requested: $numBytes") // Add this task to the taskMemory map just so we can keep an accurate count of the number @@ -80,7 +80,7 @@ class ExecutionMemoryPool( if (!memoryForTask.contains(taskAttemptId)) { memoryForTask(taskAttemptId) = 0L // This will later cause waiting tasks to wake up and check numTasks again - memoryManager.notifyAll() + lock.notifyAll() } // Keep looping until we're either sure that we don't want to grant this request (because this @@ -108,7 +108,7 @@ class ExecutionMemoryPool( } else { logInfo( s"TID $taskAttemptId waiting for at least 1/2N of $poolName pool to be free") - memoryManager.wait() + lock.wait() } } else { memoryForTask(taskAttemptId) += toGrant @@ -121,7 +121,7 @@ class ExecutionMemoryPool( /** * Release `numBytes` of memory acquired by the given task. */ - def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = memoryManager.synchronized { + def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = lock.synchronized { val curMem = memoryForTask.getOrElse(taskAttemptId, 0L) var memoryToFree = if (curMem < numBytes) { logWarning( @@ -137,14 +137,14 @@ class ExecutionMemoryPool( memoryForTask.remove(taskAttemptId) } } - memoryManager.notifyAll() // Notify waiters in acquireMemory() that memory has been freed + lock.notifyAll() // Notify waiters in acquireMemory() that memory has been freed } /** * Release all memory for the given task and mark it as inactive (e.g. when a task ends). * @return the number of bytes freed. */ - def releaseAllMemoryForTask(taskAttemptId: Long): Long = memoryManager.synchronized { + def releaseAllMemoryForTask(taskAttemptId: Long): Long = lock.synchronized { val numBytesToFree = getMemoryUsageForTask(taskAttemptId) releaseMemory(numBytesToFree, taskAttemptId) numBytesToFree diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index ebed719ab663c..463a801bfba15 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -36,6 +36,7 @@ import org.apache.spark.unsafe.memory.MemoryAllocator private[spark] abstract class MemoryManager( conf: SparkConf, numCores: Int, + initialStorageMemory: Long, maxOnHeapExecutionMemory: Long) extends Logging { // -- Methods related to memory allocation policies and bookkeeping ------------------------------ @@ -47,6 +48,8 @@ private[spark] abstract class MemoryManager( @GuardedBy("this") protected val offHeapExecutionMemoryPool = new ExecutionMemoryPool(this, "off-heap execution") + storageMemoryPool.incrementPoolSize(initialStorageMemory) + onHeapExecutionMemoryPool.incrementPoolSize(maxOnHeapExecutionMemory) offHeapExecutionMemoryPool.incrementPoolSize(conf.getSizeAsBytes("spark.memory.offHeapSize", 0)) /** @@ -181,6 +184,14 @@ private[spark] abstract class MemoryManager( // -- Fields related to Tungsten managed memory ------------------------------------------------- + /** + * Tracks whether Tungsten memory will be allocated on the JVM heap or off-heap using + * sun.misc.Unsafe. + */ + final val tungstenMemoryMode: MemoryMode = { + if (conf.getBoolean("spark.unsafe.offHeap", false)) MemoryMode.OFF_HEAP else MemoryMode.ON_HEAP + } + /** * The default page size, in bytes. * @@ -194,19 +205,15 @@ private[spark] abstract class MemoryManager( val cores = if (numCores > 0) numCores else Runtime.getRuntime.availableProcessors() // Because of rounding to next power of 2, we may have safetyFactor as 8 in worst case val safetyFactor = 16 - val size = ByteArrayMethods.nextPowerOf2(maxOnHeapExecutionMemory / cores / safetyFactor) + val maxTungstenMemory: Long = tungstenMemoryMode match { + case MemoryMode.ON_HEAP => maxOnHeapExecutionMemory + case MemoryMode.OFF_HEAP => offHeapExecutionMemoryPool.poolSize + } + val size = ByteArrayMethods.nextPowerOf2(maxTungstenMemory / cores / safetyFactor) val default = math.min(maxPageSize, math.max(minPageSize, size)) conf.getSizeAsBytes("spark.buffer.pageSize", default) } - /** - * Tracks whether Tungsten memory will be allocated on the JVM heap or off-heap using - * sun.misc.Unsafe. - */ - final val tungstenMemoryMode: MemoryMode = { - if (conf.getBoolean("spark.unsafe.offHeap", false)) MemoryMode.OFF_HEAP else MemoryMode.ON_HEAP - } - /** * Allocates memory for use by Unsafe/Tungsten code. */ diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala index 320388aade94f..503d8da66c3d6 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala @@ -23,33 +23,33 @@ import javax.annotation.concurrent.GuardedBy * Manages bookkeeping for an adjustable-sized region of memory. This class is internal to * the [[MemoryManager]]. See subclasses for more details. * - * @param memoryManager a [[MemoryManager]] instance, used for synchronization. We purposely - * erase the type to `Object` to avoid programming errors, since this object - * should only be used for synchronization purposes. + * @param lock a [[MemoryManager]] instance, used for synchronization. We purposely erase the type + * to `Object` to avoid programming errors, since this object should only be used for + * synchronization purposes. */ -abstract class MemoryPool(memoryManager: Object) { +abstract class MemoryPool(lock: Object) { - @GuardedBy("memoryManager") + @GuardedBy("lcok") private[this] var _poolSize: Long = 0 /** * Returns the current size of the pool, in bytes. */ - final def poolSize: Long = memoryManager.synchronized { + final def poolSize: Long = lock.synchronized { _poolSize } /** * Returns the amount of free memory in the pool, in bytes. */ - final def memoryFree: Long = memoryManager.synchronized { + final def memoryFree: Long = lock.synchronized { _poolSize - memoryUsed } /** * Expands the pool by `delta` bytes. */ - final def incrementPoolSize(delta: Long): Unit = memoryManager.synchronized { + final def incrementPoolSize(delta: Long): Unit = lock.synchronized { require(delta >= 0) _poolSize += delta } @@ -57,7 +57,7 @@ abstract class MemoryPool(memoryManager: Object) { /** * Shrinks the pool by `delta` bytes. */ - final def decrementPoolSize(delta: Long): Unit = memoryManager.synchronized { + final def decrementPoolSize(delta: Long): Unit = lock.synchronized { require(delta >= 0) require(delta <= _poolSize) require(_poolSize - delta >= memoryUsed) diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala index dfc076b51e19e..c3aff730293df 100644 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala @@ -34,7 +34,11 @@ private[spark] class StaticMemoryManager( maxOnHeapExecutionMemory: Long, override val maxStorageMemory: Long, numCores: Int) - extends MemoryManager(conf, numCores, maxOnHeapExecutionMemory) { + extends MemoryManager( + conf, + numCores, + initialStorageMemory = maxStorageMemory, + maxOnHeapExecutionMemory = maxOnHeapExecutionMemory) { def this(conf: SparkConf, numCores: Int) { this( @@ -44,9 +48,6 @@ private[spark] class StaticMemoryManager( numCores) } - onHeapExecutionMemoryPool.incrementPoolSize(maxOnHeapExecutionMemory) - storageMemoryPool.incrementPoolSize(maxStorageMemory) - // Max number of bytes worth of blocks to evict when unrolling private val maxMemoryToEvictForUnroll: Long = { (maxStorageMemory * conf.getDouble("spark.storage.unrollFraction", 0.2)).toLong diff --git a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala index c217f019957d5..6a322eabf81ed 100644 --- a/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala @@ -29,14 +29,14 @@ import org.apache.spark.storage.{MemoryStore, BlockStatus, BlockId} * Performs bookkeeping for managing an adjustable-size pool of memory that is used for storage * (caching). * - * @param memoryManager a [[MemoryManager]] instance to synchronize on + * @param lock a [[MemoryManager]] instance to synchronize on */ -class StorageMemoryPool(memoryManager: Object) extends MemoryPool(memoryManager) with Logging { +class StorageMemoryPool(lock: Object) extends MemoryPool(lock) with Logging { - @GuardedBy("memoryManager") + @GuardedBy("lock") private[this] var _memoryUsed: Long = 0L - override def memoryUsed: Long = memoryManager.synchronized { + override def memoryUsed: Long = lock.synchronized { _memoryUsed } @@ -64,7 +64,7 @@ class StorageMemoryPool(memoryManager: Object) extends MemoryPool(memoryManager) def acquireMemory( blockId: BlockId, numBytes: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = memoryManager.synchronized { + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = lock.synchronized { acquireMemory(blockId, numBytes, numBytes, evictedBlocks) } @@ -80,7 +80,7 @@ class StorageMemoryPool(memoryManager: Object) extends MemoryPool(memoryManager) blockId: BlockId, numBytesToAcquire: Long, numBytesToFree: Long, - evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = memoryManager.synchronized { + evictedBlocks: mutable.Buffer[(BlockId, BlockStatus)]): Boolean = lock.synchronized { assert(numBytesToAcquire >= 0) assert(numBytesToFree >= 0) assert(memoryUsed <= poolSize) @@ -101,7 +101,7 @@ class StorageMemoryPool(memoryManager: Object) extends MemoryPool(memoryManager) enoughMemory } - def releaseMemory(size: Long): Unit = memoryManager.synchronized { + def releaseMemory(size: Long): Unit = lock.synchronized { if (size > _memoryUsed) { logWarning(s"Attempted to release $size bytes of storage " + s"memory when we only have ${_memoryUsed} bytes") @@ -111,7 +111,7 @@ class StorageMemoryPool(memoryManager: Object) extends MemoryPool(memoryManager) } } - def releaseAllMemory(): Unit = memoryManager.synchronized { + def releaseAllMemory(): Unit = lock.synchronized { _memoryUsed = 0 } @@ -119,7 +119,7 @@ class StorageMemoryPool(memoryManager: Object) extends MemoryPool(memoryManager) * Try to shrink the size of this storage memory pool by `spaceToFree` bytes. Return the number * of bytes removed from the pool's capacity. */ - def shrinkPoolToFreeSpace(spaceToFree: Long): Long = memoryManager.synchronized { + def shrinkPoolToFreeSpace(spaceToFree: Long): Long = lock.synchronized { // First, shrink the pool by reclaiming free memory: val spaceFreedByReleasingUnusedMemory = Math.min(spaceToFree, memoryFree) decrementPoolSize(spaceFreedByReleasingUnusedMemory) diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 84cbcaf40c98c..6cc8f9cd3affd 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -54,13 +54,11 @@ private[spark] class UnifiedMemoryManager private[memory] ( extends MemoryManager( conf, numCores, - // TODO(josh): it is confusing how this interacts with page size calculations: + initialStorageMemory = storageRegionSize, maxOnHeapExecutionMemory = maxMemory - storageRegionSize) { - // We always maintain the invariant that - // onHeapExecutionMemoryPool.poolSize + storageMemoryPool.poolSize == maxMemory - // At first, all memory is allocated towards execution: - onHeapExecutionMemoryPool.incrementPoolSize(maxMemory) + // We always maintain this invariant: + assert(onHeapExecutionMemoryPool.poolSize + storageMemoryPool.poolSize == maxMemory) override def maxStorageMemory: Long = synchronized { maxMemory - onHeapExecutionMemoryPool.memoryUsed @@ -88,8 +86,8 @@ private[spark] class UnifiedMemoryManager private[memory] ( // storage. We can reclaim any free memory from the storage pool. If the storage pool // has grown to become larger than `storageRegionSize`, we can evict blocks and reclaim // the memory that storage has borrowed from execution. - val memoryReclaimableFromStorage = - math.max(storageMemoryPool.memoryFree, storageMemoryPool.poolSize - storageRegionSize) + val memoryReclaimableFromStorage = storageMemoryPool.memoryFree + + math.max(storageMemoryPool.memoryUsed - storageRegionSize, 0) if (memoryReclaimableFromStorage > 0) { // Only reclaim as much space as is necessary and available: val spaceReclaimed = storageMemoryPool.shrinkPoolToFreeSpace( diff --git a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala index 51018d7066c75..a41d63fbce7d6 100644 --- a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala +++ b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala @@ -23,12 +23,10 @@ import org.apache.spark.SparkConf import org.apache.spark.storage.{BlockStatus, BlockId} class TestMemoryManager(conf: SparkConf) - extends MemoryManager(conf, numCores = 1, maxOnHeapExecutionMemory = Long.MaxValue) { - - storageMemoryPool.incrementPoolSize(Long.MaxValue) - onHeapExecutionMemoryPool.incrementPoolSize(Long.MaxValue) - - // TODO(josh): separate configs for available on- and off-heap + extends MemoryManager( + conf, numCores = 1, + initialStorageMemory = Long.MaxValue, + maxOnHeapExecutionMemory = Long.MaxValue) { override private[memory] def acquireExecutionMemory( numBytes: Long, diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index 3fb387ca92b98..8cebe81c3bfff 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -127,7 +127,8 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes assertEnsureFreeSpaceNotCalled(ms) // Execution wants 200 bytes but only 150 are free, so storage is evicted assert(mm.acquireExecutionMemory(200L, taskAttemptId, MemoryMode.ON_HEAP) === 200L) - assertEnsureFreeSpaceCalled(ms, 200L) + assert(mm.executionMemoryUsed === 300L) + assertEnsureFreeSpaceCalled(ms, 50L) assert(mm.executionMemoryUsed === 300L) mm.releaseAllStorageMemory() require(mm.executionMemoryUsed === 300L) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index d8bf300e05995..8c9b9c85e37fc 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -23,7 +23,6 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.spark.TaskContext; -import org.apache.spark.memory.MemoryMode; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; import org.apache.spark.sql.catalyst.expressions.codegen.BaseOrdering; From eac53f16c78f01b413551dd8f277605511b8257b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 6 Nov 2015 14:18:56 -0800 Subject: [PATCH 29/31] Minor update to calculation of page sizes. --- .../scala/org/apache/spark/memory/MemoryManager.scala | 10 +++++----- .../org/apache/spark/memory/StaticMemoryManager.scala | 4 ++-- .../org/apache/spark/memory/UnifiedMemoryManager.scala | 4 ++-- .../org/apache/spark/memory/TestMemoryManager.scala | 5 +---- 4 files changed, 10 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 463a801bfba15..ceb8ea434e1be 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -36,8 +36,8 @@ import org.apache.spark.unsafe.memory.MemoryAllocator private[spark] abstract class MemoryManager( conf: SparkConf, numCores: Int, - initialStorageMemory: Long, - maxOnHeapExecutionMemory: Long) extends Logging { + storageMemory: Long, + onHeapExecutionMemory: Long) extends Logging { // -- Methods related to memory allocation policies and bookkeeping ------------------------------ @@ -48,8 +48,8 @@ private[spark] abstract class MemoryManager( @GuardedBy("this") protected val offHeapExecutionMemoryPool = new ExecutionMemoryPool(this, "off-heap execution") - storageMemoryPool.incrementPoolSize(initialStorageMemory) - onHeapExecutionMemoryPool.incrementPoolSize(maxOnHeapExecutionMemory) + storageMemoryPool.incrementPoolSize(storageMemory) + onHeapExecutionMemoryPool.incrementPoolSize(onHeapExecutionMemory) offHeapExecutionMemoryPool.incrementPoolSize(conf.getSizeAsBytes("spark.memory.offHeapSize", 0)) /** @@ -206,7 +206,7 @@ private[spark] abstract class MemoryManager( // Because of rounding to next power of 2, we may have safetyFactor as 8 in worst case val safetyFactor = 16 val maxTungstenMemory: Long = tungstenMemoryMode match { - case MemoryMode.ON_HEAP => maxOnHeapExecutionMemory + case MemoryMode.ON_HEAP => onHeapExecutionMemoryPool.poolSize case MemoryMode.OFF_HEAP => offHeapExecutionMemoryPool.poolSize } val size = ByteArrayMethods.nextPowerOf2(maxTungstenMemory / cores / safetyFactor) diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala index c3aff730293df..12a094306861f 100644 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala @@ -37,8 +37,8 @@ private[spark] class StaticMemoryManager( extends MemoryManager( conf, numCores, - initialStorageMemory = maxStorageMemory, - maxOnHeapExecutionMemory = maxOnHeapExecutionMemory) { + maxStorageMemory, + maxOnHeapExecutionMemory) { def this(conf: SparkConf, numCores: Int) { this( diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 6cc8f9cd3affd..3a146f1e6ee46 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -54,8 +54,8 @@ private[spark] class UnifiedMemoryManager private[memory] ( extends MemoryManager( conf, numCores, - initialStorageMemory = storageRegionSize, - maxOnHeapExecutionMemory = maxMemory - storageRegionSize) { + storageRegionSize, + maxMemory - storageRegionSize) { // We always maintain this invariant: assert(onHeapExecutionMemoryPool.poolSize + storageMemoryPool.poolSize == maxMemory) diff --git a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala index a41d63fbce7d6..0706a6e45de8f 100644 --- a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala +++ b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala @@ -23,10 +23,7 @@ import org.apache.spark.SparkConf import org.apache.spark.storage.{BlockStatus, BlockId} class TestMemoryManager(conf: SparkConf) - extends MemoryManager( - conf, numCores = 1, - initialStorageMemory = Long.MaxValue, - maxOnHeapExecutionMemory = Long.MaxValue) { + extends MemoryManager(conf, numCores = 1, Long.MaxValue, Long.MaxValue) { override private[memory] def acquireExecutionMemory( numBytes: Long, From 55feee0cbe5f0f746006ed1ef4377f967ac20f8a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 6 Nov 2015 14:35:09 -0800 Subject: [PATCH 30/31] Correct previous bug fix, which misidentified problem. --- .../org/apache/spark/memory/UnifiedMemoryManager.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 3a146f1e6ee46..8be5b05419094 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -82,16 +82,17 @@ private[spark] class UnifiedMemoryManager private[memory] ( memoryMode match { case MemoryMode.ON_HEAP => if (numBytes > onHeapExecutionMemoryPool.memoryFree) { + val extraMemoryNeeded = numBytes - onHeapExecutionMemoryPool.memoryFree // There is not enough free memory in the execution pool, so try to reclaim memory from // storage. We can reclaim any free memory from the storage pool. If the storage pool // has grown to become larger than `storageRegionSize`, we can evict blocks and reclaim // the memory that storage has borrowed from execution. - val memoryReclaimableFromStorage = storageMemoryPool.memoryFree + - math.max(storageMemoryPool.memoryUsed - storageRegionSize, 0) + val memoryReclaimableFromStorage = + math.max(storageMemoryPool.memoryFree, storageMemoryPool.poolSize - storageRegionSize) if (memoryReclaimableFromStorage > 0) { // Only reclaim as much space as is necessary and available: val spaceReclaimed = storageMemoryPool.shrinkPoolToFreeSpace( - math.min(numBytes, memoryReclaimableFromStorage)) + math.min(extraMemoryNeeded, memoryReclaimableFromStorage)) onHeapExecutionMemoryPool.incrementPoolSize(spaceReclaimed) } } From 1e5eefac94a2db9b575bd391c3836b9352fdaee0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 6 Nov 2015 14:50:34 -0800 Subject: [PATCH 31/31] Fix typo. --- core/src/main/scala/org/apache/spark/memory/MemoryPool.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala index 503d8da66c3d6..bfeec47e3892e 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryPool.scala @@ -29,7 +29,7 @@ import javax.annotation.concurrent.GuardedBy */ abstract class MemoryPool(lock: Object) { - @GuardedBy("lcok") + @GuardedBy("lock") private[this] var _poolSize: Long = 0 /**