-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-9419] ShuffleMemoryManager and MemoryStore should track memory on a per-task, not per-thread, basis #7734
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
11 commits
Select commit
Hold shift + click to select a range
c9e8e54
Use TaskAttemptIds to track unroll memory
JoshRosen 2e1e0f8
Use TaskAttemptIds to track shuffle memory
JoshRosen 1b0083b
Roll back fix in PySpark, which is no longer necessary
JoshRosen 5e2f01e
Fix capitalization
JoshRosen fa78ee8
Move Executor's cleanup into Task so that TaskContext is defined when…
JoshRosen f57f3f2
More thread -> task changes
JoshRosen 7b0f04b
Fix ShuffleMemoryManagerSuite
JoshRosen 44f6497
Fix long line.
JoshRosen ed25d3b
Address minor PR review comments
JoshRosen 57c9b4e
Merge remote-tracking branch 'origin/master' into memory-tracking-fixes
JoshRosen b4b1702
Propagate TaskContext to writer threads.
JoshRosen File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,95 +19,101 @@ package org.apache.spark.shuffle | |
|
|
||
| import scala.collection.mutable | ||
|
|
||
| import org.apache.spark.{Logging, SparkException, SparkConf} | ||
| import org.apache.spark.{Logging, SparkException, SparkConf, TaskContext} | ||
|
|
||
| /** | ||
| * Allocates a pool of memory to task threads for use in shuffle operations. Each disk-spilling | ||
| * Allocates a pool of memory to tasks for use in shuffle operations. Each disk-spilling | ||
| * collection (ExternalAppendOnlyMap or ExternalSorter) used by these tasks can acquire memory | ||
| * from this pool and release it as it spills data out. When a task ends, all its memory will be | ||
| * released by the Executor. | ||
| * | ||
| * This class tries to ensure that each thread gets a reasonable share of memory, instead of some | ||
| * thread ramping up to a large amount first and then causing others to spill to disk repeatedly. | ||
| * If there are N threads, it ensures that each thread can acquire at least 1 / 2N of the memory | ||
| * This class 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 tasks can acquire at least 1 / 2N of the memory | ||
| * before it has to spill, and at most 1 / N. Because N varies dynamically, we keep track of the | ||
| * set of active threads and redo the calculations of 1 / 2N and 1 / N in waiting threads whenever | ||
| * 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 on "this" to mutate state and using | ||
| * wait() and notifyAll() to signal changes. | ||
| */ | ||
| private[spark] class ShuffleMemoryManager(maxMemory: Long) extends Logging { | ||
| private val threadMemory = new mutable.HashMap[Long, Long]() // threadId -> memory bytes | ||
| private val taskMemory = new mutable.HashMap[Long, Long]() // taskAttemptId -> memory bytes | ||
|
|
||
| def this(conf: SparkConf) = this(ShuffleMemoryManager.getMaxMemory(conf)) | ||
|
|
||
| private def currentTaskAttemptId(): Long = { | ||
| // In case this is called on the driver, return an invalid task attempt id. | ||
| Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. add a line explaining why the default value is needed |
||
| } | ||
|
|
||
| /** | ||
| * Try to acquire up to numBytes memory for the current thread, and return the number of bytes | ||
| * Try to acquire up to numBytes 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 thread has a chance to ramp up to at least 1 / 2N of the | ||
| * total memory pool (where N is the # of active threads) before it is forced to spill. This can | ||
| * happen if the number of threads increases but an older thread had a lot of memory already. | ||
| * 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 increases but an older task had a lot of memory already. | ||
| */ | ||
| def tryToAcquire(numBytes: Long): Long = synchronized { | ||
| val threadId = Thread.currentThread().getId | ||
| val taskAttemptId = currentTaskAttemptId() | ||
| assert(numBytes > 0, "invalid number of bytes requested: " + numBytes) | ||
|
|
||
| // Add this thread to the threadMemory map just so we can keep an accurate count of the number | ||
| // of active threads, to let other threads ramp down their memory in calls to tryToAcquire | ||
| if (!threadMemory.contains(threadId)) { | ||
| threadMemory(threadId) = 0L | ||
| notifyAll() // Will later cause waiting threads to wake up and check numThreads again | ||
| // 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 (!taskMemory.contains(taskAttemptId)) { | ||
| taskMemory(taskAttemptId) = 0L | ||
| notifyAll() // Will later cause waiting tasks to wake up and check numThreads again | ||
| } | ||
|
|
||
| // Keep looping until we're either sure that we don't want to grant this request (because this | ||
| // thread would have more than 1 / numActiveThreads of the memory) or we have enough free | ||
| // memory to give it (we always let each thread get at least 1 / (2 * numActiveThreads)). | ||
| // 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)). | ||
| while (true) { | ||
| val numActiveThreads = threadMemory.keys.size | ||
| val curMem = threadMemory(threadId) | ||
| val freeMemory = maxMemory - threadMemory.values.sum | ||
| val numActiveTasks = taskMemory.keys.size | ||
| val curMem = taskMemory(taskAttemptId) | ||
| val freeMemory = maxMemory - taskMemory.values.sum | ||
|
|
||
| // How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads; | ||
| // 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 / numActiveThreads) - curMem)) | ||
| val maxToGrant = math.min(numBytes, math.max(0, (maxMemory / numActiveTasks) - curMem)) | ||
|
|
||
| if (curMem < maxMemory / (2 * numActiveThreads)) { | ||
| // We want to let each thread get at least 1 / (2 * numActiveThreads) before blocking; | ||
| // if we can't give it this much now, wait for other threads to free up memory | ||
| // (this happens if older threads allocated lots of memory before N grew) | ||
| if (freeMemory >= math.min(maxToGrant, maxMemory / (2 * numActiveThreads) - curMem)) { | ||
| 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)) { | ||
| val toGrant = math.min(maxToGrant, freeMemory) | ||
| threadMemory(threadId) += toGrant | ||
| taskMemory(taskAttemptId) += toGrant | ||
| return toGrant | ||
| } else { | ||
| logInfo(s"Thread $threadId waiting for at least 1/2N of shuffle memory pool to be free") | ||
| logInfo( | ||
| s"Thread $taskAttemptId waiting for at least 1/2N of shuffle memory pool to be free") | ||
| wait() | ||
| } | ||
| } else { | ||
| // Only give it as much memory as is free, which might be none if it reached 1 / numThreads | ||
| val toGrant = math.min(maxToGrant, freeMemory) | ||
| threadMemory(threadId) += toGrant | ||
| taskMemory(taskAttemptId) += toGrant | ||
| return toGrant | ||
| } | ||
| } | ||
| 0L // Never reached | ||
| } | ||
|
|
||
| /** Release numBytes bytes for the current thread. */ | ||
| /** Release numBytes bytes for the current task. */ | ||
| def release(numBytes: Long): Unit = synchronized { | ||
| val threadId = Thread.currentThread().getId | ||
| val curMem = threadMemory.getOrElse(threadId, 0L) | ||
| val taskAttemptId = currentTaskAttemptId() | ||
| val curMem = taskMemory.getOrElse(taskAttemptId, 0L) | ||
| if (curMem < numBytes) { | ||
| throw new SparkException( | ||
| s"Internal error: release called on ${numBytes} bytes but thread only has ${curMem}") | ||
| s"Internal error: release called on ${numBytes} bytes but task only has ${curMem}") | ||
| } | ||
| threadMemory(threadId) -= numBytes | ||
| taskMemory(taskAttemptId) -= numBytes | ||
| notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has been freed | ||
| } | ||
|
|
||
| /** Release all memory for the current thread and mark it as inactive (e.g. when a task ends). */ | ||
| def releaseMemoryForThisThread(): Unit = synchronized { | ||
| val threadId = Thread.currentThread().getId | ||
| threadMemory.remove(threadId) | ||
| /** Release all memory for the current task and mark it as inactive (e.g. when a task ends). */ | ||
| def releaseMemoryForThisTask(): Unit = synchronized { | ||
| val taskAttemptId = currentTaskAttemptId() | ||
| taskMemory.remove(taskAttemptId) | ||
| notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has been freed | ||
| } | ||
| } | ||
|
|
||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
/cc @davies for this PySpark change.