Github user rxin commented on a diff in the pull request:
https://github.com/apache/spark/pull/7734#discussion_r35719677
--- Diff:
core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala ---
@@ -19,95 +19,100 @@ 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 = {
+ Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L)
--- End diff --
add a line explaining why the default value is needed
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]