Skip to content
Closed
Changes from 1 commit
Commits
Show all changes
46 commits
Select commit Hold shift + click to select a range
b7c9c23
Move Unsafe mem. mgrs. to spark-core subproject.
JoshRosen Oct 14, 2015
25ba4b5
Merge ExecutorMemoryManager into MemoryManager.
JoshRosen Oct 14, 2015
3d997ce
Naming and formatting fixes.
JoshRosen Oct 16, 2015
d9e6b84
Move Tungsten-related methods to end of MemoryManager file.
JoshRosen Oct 16, 2015
98ef86b
Add taskAttemptId to TaskMemoryManager constructor.
JoshRosen Oct 16, 2015
8f93e94
Move ShuffleMemoryManager into memory package.
JoshRosen Oct 16, 2015
3bbc54d
Merge remote-tracking branch 'origin/master' into SPARK-10984
JoshRosen Oct 16, 2015
88a7970
Fix bug in AbstractBytesToBytesMapSuite.
JoshRosen Oct 16, 2015
ec48ff9
Refactor the existing Tungsten TaskMemoryManager interactions so Tung…
JoshRosen Oct 16, 2015
6f98bc4
Move TaskMemoryManager from unsafe to memory.
JoshRosen Oct 16, 2015
6459397
Further minimization of ShuffleMemoryManager usage.
JoshRosen Oct 16, 2015
60c66b2
Merge ShuffleMemoryManager into MemoryManager.
JoshRosen Oct 17, 2015
7d6a37f
Clean up interaction between TaskMemoryManager and MemoryManager.
JoshRosen Oct 17, 2015
0dc21dc
Merge remote-tracking branch 'origin/master' into SPARK-10984
JoshRosen Oct 22, 2015
f21b767
Fix compilation.
JoshRosen Oct 22, 2015
46ad693
Fix Scalastyle
JoshRosen Oct 22, 2015
c33e330
Fix import ordering in Executor.scala
JoshRosen Oct 22, 2015
ef45d91
Fix import ordering in Task.scala
JoshRosen Oct 22, 2015
c7eac69
Fix import ordering in TaskContextImpl
JoshRosen Oct 22, 2015
d86f435
Fix spillable collection tests
JoshRosen Oct 22, 2015
bba5550
Integrate TaskMemoryManager acquire/releasePage with MemoryManager bo…
JoshRosen Oct 22, 2015
66ae259
Move pooling logic into allocators themselves.
JoshRosen Oct 22, 2015
b1d5151
Scaladoc updates.
JoshRosen Oct 22, 2015
d0c0dd9
Update Spillable to properly integrate with TaskMemoryManager.
JoshRosen Oct 22, 2015
48149fc
Move pageSizeBytes to Tungsten section
JoshRosen Oct 23, 2015
c8ba196
Cleanup after merging of ShuffleMemoryManager into MemoryManager.
JoshRosen Oct 23, 2015
63a6cbc
Rename getMemoryConsumptionForThisTask to getExecutionMemoryUsageForTask
JoshRosen Oct 23, 2015
6ec9c30
Properly thread numCores to memory manager.
JoshRosen Oct 23, 2015
1593fad
Explain why MemoryBlock.pageNumber is public
JoshRosen Oct 23, 2015
64bec0b
Fix TaskMemoryManagerSuite tests.
JoshRosen Oct 23, 2015
f9240e9
Fix compilation
JoshRosen Oct 23, 2015
a95bc08
Fix a memory leak in UnsafeShuffleWriter's sorter
JoshRosen Oct 23, 2015
b3ad761
Remove println
JoshRosen Oct 23, 2015
a7e8320
Fix Scalastyle.
JoshRosen Oct 23, 2015
e874a45
Fix remaining TODOs in UnsafeShuffleWriterSuite.
JoshRosen Oct 23, 2015
2ba6e51
Fix DeveloperAPI change
JoshRosen Oct 23, 2015
0c13723
Address comments in MemoryManager
JoshRosen Oct 23, 2015
04ec429
Release memory acquired after unsuccessful allocatePage() call
JoshRosen Oct 23, 2015
e56d039
Fix EAOM compilation.
JoshRosen Oct 23, 2015
aa14113
Port tests from ShuffleMemoryManagerSuite
JoshRosen Oct 23, 2015
7addf8b
Remove unused non-page-memory allocation methods.
JoshRosen Oct 23, 2015
5af0b17
Update Tungsten tests
JoshRosen Oct 23, 2015
a264703
Fix execution memory leaks in Spillable collections
JoshRosen Oct 24, 2015
f2ab708
Fix NPE in UnsafeRowSerializerSuite
JoshRosen Oct 24, 2015
0b5c72f
Update EAOM tests to reflect fact that iterator() is destructive.
JoshRosen Oct 24, 2015
f68fdb1
Fix streaming test compilation
JoshRosen Oct 26, 2015
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Address comments in MemoryManager
  • Loading branch information
JoshRosen committed Oct 23, 2015
commit 0c137236a4474c6f409682e51a68415d753e054c
40 changes: 22 additions & 18 deletions core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte
@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 memoryConsumptionForTask = new mutable.HashMap[Long, Long]()
@GuardedBy("this") private val executionMemoryForTask = new mutable.HashMap[Long, Long]()

/**
* Set the [[MemoryStore]] used by this manager to evict cached blocks.
Expand Down Expand Up @@ -130,14 +130,18 @@ 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.
*/
def acquireExecutionMemory(numBytes: Long, taskAttemptId: Long): Long = synchronized {
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 (!memoryConsumptionForTask.contains(taskAttemptId)) {
memoryConsumptionForTask(taskAttemptId) = 0L
if (!executionMemoryForTask.contains(taskAttemptId)) {
executionMemoryForTask(taskAttemptId) = 0L
// This will later cause waiting tasks to wake up and check numTasks again
notifyAll()
}
Expand All @@ -154,7 +158,7 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte
val lastUpdatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]())
metrics.updatedBlocks = Some(lastUpdatedBlocks ++ evictedBlocks.toSeq)
}
memoryConsumptionForTask(taskAttemptId) += acquired
executionMemoryForTask(taskAttemptId) += acquired
acquired
}

Expand All @@ -163,9 +167,9 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte
// 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 = memoryConsumptionForTask.keys.size
val curMem = memoryConsumptionForTask(taskAttemptId)
val freeMemory = maxExecutionMemory - memoryConsumptionForTask.values.sum
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
Expand Down Expand Up @@ -207,20 +211,21 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte
/**
* Release numBytes of execution memory belonging to the given task.
*/
private[memory]
final def releaseExecutionMemory(numBytes: Long, taskAttemptId: Long): Unit = synchronized {
val curMem = memoryConsumptionForTask.getOrElse(taskAttemptId, 0L)
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 (memoryConsumptionForTask.contains(taskAttemptId)) {
memoryConsumptionForTask(taskAttemptId) -= numBytes
if (memoryConsumptionForTask(taskAttemptId) <= 0) {
memoryConsumptionForTask.remove(taskAttemptId)
if (executionMemoryForTask.contains(taskAttemptId)) {
executionMemoryForTask(taskAttemptId) -= numBytes
if (executionMemoryForTask(taskAttemptId) <= 0) {
executionMemoryForTask.remove(taskAttemptId)
}
releaseExecutionMemory(numBytes)
}
notifyAll() // Notify waiters in tryToAcquire that memory has been freed
notifyAll() // Notify waiters in acquireExecutionMemory() that memory has been freed
}

/**
Expand Down Expand Up @@ -278,10 +283,10 @@ 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 {
memoryConsumptionForTask.getOrElse(taskAttemptId, 0L)
executionMemoryForTask.getOrElse(taskAttemptId, 0L)
}

// -- Methods related to Tungsten managed memory -------------------------------------------------
// -- Fields related to Tungsten managed memory -------------------------------------------------

/**
* The default page size, in bytes.
Expand Down Expand Up @@ -309,8 +314,7 @@ private[spark] abstract class MemoryManager(conf: SparkConf, numCores: Int) exte
!conf.getBoolean("spark.unsafe.offHeap", false)

/**
* Allocates memory for use by Unsafe/Tungsten code. Exposed to enable untracked allocations of
* temporary data structures.
* Allocates memory for use by Unsafe/Tungsten code.
*/
private[memory] final val tungstenMemoryAllocator: MemoryAllocator =
if (tungstenMemoryIsAllocatedInHeap) MemoryAllocator.HEAP else MemoryAllocator.UNSAFE
Expand Down