-
Notifications
You must be signed in to change notification settings - Fork 176
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
feat: Add a spark.comet.exec.memoryPool
configuration for experimenting with various datafusion memory pool setups.
#1021
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||||
---|---|---|---|---|---|---|---|---|
|
@@ -23,7 +23,7 @@ import org.apache.spark._ | |||||||
import org.apache.spark.sql.comet.CometMetricNode | ||||||||
import org.apache.spark.sql.vectorized._ | ||||||||
|
||||||||
import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_FRACTION, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} | ||||||||
import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_FRACTION, COMET_EXEC_MEMORY_POOL_TYPE, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS} | ||||||||
import org.apache.comet.vector.NativeUtil | ||||||||
|
||||||||
/** | ||||||||
|
@@ -72,8 +72,11 @@ class CometExecIterator( | |||||||
new CometTaskMemoryManager(id), | ||||||||
batchSize = COMET_BATCH_SIZE.get(), | ||||||||
use_unified_memory_manager = conf.getBoolean("spark.memory.offHeap.enabled", false), | ||||||||
memory_pool_type = COMET_EXEC_MEMORY_POOL_TYPE.get(), | ||||||||
memory_limit = CometSparkSessionExtensions.getCometMemoryOverhead(conf), | ||||||||
memory_limit_per_task = getMemoryLimitPerTask(conf), | ||||||||
memory_fraction = COMET_EXEC_MEMORY_FRACTION.get(), | ||||||||
task_attempt_id = TaskContext.get().taskAttemptId, | ||||||||
debug = COMET_DEBUG_ENABLED.get(), | ||||||||
explain = COMET_EXPLAIN_NATIVE_ENABLED.get(), | ||||||||
workerThreads = COMET_WORKER_THREADS.get(), | ||||||||
|
@@ -84,6 +87,30 @@ class CometExecIterator( | |||||||
private var currentBatch: ColumnarBatch = null | ||||||||
private var closed: Boolean = false | ||||||||
|
||||||||
private def getMemoryLimitPerTask(conf: SparkConf): Long = { | ||||||||
val numCores = numDriverOrExecutorCores(conf).toFloat | ||||||||
val maxMemory = CometSparkSessionExtensions.getCometMemoryOverhead(conf) | ||||||||
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. We should be using the total executor memory here rather than just the overhead memory. Without this change I was unable to run any benchmarks.
Suggested change
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. I thought that overhead memory is the amount of native memory reserved for comet to use. The actual memory usage will be doubled if we reserve the same amount of executor memory for comet. 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. That's a good point. The default overhead amount was way too low for the benchmarks to run. Using the executor amount is probably overkill. I'll give this some more thought. |
||||||||
val coresPerTask = conf.get("spark.task.cpus", "1").toFloat | ||||||||
// example 16GB maxMemory * 16 cores with 4 cores per task results | ||||||||
// in memory_limit_per_task = 16 GB * 4 / 16 = 16 GB / 4 = 4GB | ||||||||
(maxMemory.toFloat * coresPerTask / numCores).toLong | ||||||||
} | ||||||||
|
||||||||
private def numDriverOrExecutorCores(conf: SparkConf): Int = { | ||||||||
def convertToInt(threads: String): Int = { | ||||||||
if (threads == "*") Runtime.getRuntime.availableProcessors() else threads.toInt | ||||||||
} | ||||||||
val LOCAL_N_REGEX = """local\[([0-9]+|\*)\]""".r | ||||||||
val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+|\*)\s*,\s*([0-9]+)\]""".r | ||||||||
val master = conf.get("spark.master") | ||||||||
master match { | ||||||||
case "local" => 1 | ||||||||
case LOCAL_N_REGEX(threads) => convertToInt(threads) | ||||||||
case LOCAL_N_FAILURES_REGEX(threads, _) => convertToInt(threads) | ||||||||
case _ => conf.get("spark.executor.cores", "1").toInt | ||||||||
} | ||||||||
} | ||||||||
|
||||||||
def getNextBatch(): Option[ColumnarBatch] = { | ||||||||
assert(partitionIndex >= 0 && partitionIndex < numParts) | ||||||||
|
||||||||
|
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.
We should add that this config only takes effect when off-heap memory is disabled.
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.
We can do that in a follow up PR though, and add some more detailed documentation in the tuning guide.
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.
Yea, I am not sure if all users can understand the difference between these pool configs. It is better we can add more description.