-
Notifications
You must be signed in to change notification settings - Fork 277
feat: Various improvements to memory pool configuration, logging, and documentation #2538
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
Changes from 21 commits
b3b0a85
191bbc4
2b1242e
f42be8f
6e987af
8ef56eb
43e83c2
ecce61e
f3051ba
f364962
edc3d0d
5f5f95e
8c8ad98
83ecd6e
07c69e7
7e4a482
648f3ee
767866b
d04d676
7fc62b6
c1a2053
e36785b
4848f64
5e812de
31e9914
2e125a1
e35f7e8
303ba75
c333959
c39afd9
4816437
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 |
|---|---|---|
|
|
@@ -517,10 +517,19 @@ object CometConf extends ShimCometConf { | |
| "'greedy_task_shared', 'fair_spill_task_shared', 'greedy_global', 'fair_spill_global', " + | ||
| "and `unbounded`. When running Spark in off-heap mode, available pool types are " + | ||
| "'greedy_unified' and `fair_unified`. The default pool type is `greedy_task_shared` " + | ||
| s"for on-heap mode and `unified` for off-heap mode. $TUNING_GUIDE.") | ||
| s"for on-heap mode and `fair_unified` for off-heap mode. $TUNING_GUIDE.") | ||
| .stringConf | ||
| .createWithDefault("default") | ||
|
|
||
| val COMET_EXEC_MEMORY_POOL_FRACTION: ConfigEntry[Double] = | ||
| conf("spark.comet.exec.memoryPool.fraction") | ||
| .doc( | ||
| "Fraction of off-heap memory pool that is available to Comet. " + | ||
| "Only applies to off-heap mode. " + | ||
| s"$TUNING_GUIDE.") | ||
| .doubleConf | ||
| .createWithDefault(1.0) | ||
|
Member
Author
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. Default is 1.0 so that this change is not a breaking change |
||
|
|
||
| val COMET_SCAN_PREFETCH_ENABLED: ConfigEntry[Boolean] = | ||
| conf("spark.comet.scan.preFetch.enabled") | ||
| .doc("Whether to enable pre-fetching feature of CometScan.") | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -45,10 +45,17 @@ and requiring shuffle memory to be separately configured. | |
| The recommended way to allocate memory for Comet is to set `spark.memory.offHeap.enabled=true`. This allows | ||
| Comet to share an off-heap memory pool with Spark, reducing the overall memory overhead. The size of the pool is | ||
| specified by `spark.memory.offHeap.size`. For more details about Spark off-heap memory mode, please refer to | ||
| Spark documentation: https://spark.apache.org/docs/latest/configuration.html. | ||
| [Spark documentation]. For full details on configuring Comet memory in off-heap mode, see the [Advanced Memory Tuning] | ||
| section of this guide. | ||
|
|
||
| [Spark documentation]: https://spark.apache.org/docs/latest/configuration.html | ||
|
|
||
| ### Configuring Comet Memory in On-Heap Mode | ||
|
|
||
| ```{warning} | ||
| Support for on-heap memory pools is deprecated and will be removed from a future release. | ||
| ``` | ||
|
|
||
| When running in on-heap mode, Comet memory can be allocated by setting `spark.comet.memoryOverhead`. If this setting | ||
| is not provided, it will be calculated by multiplying the current Spark executor memory by | ||
| `spark.comet.memory.overhead.factor` (default value is `0.2`) which may or may not result in enough memory for | ||
|
|
@@ -59,10 +66,13 @@ Comet supports native shuffle and columnar shuffle (these terms are explained in | |
| In on-heap mode, columnar shuffle memory must be separately allocated using `spark.comet.columnar.shuffle.memorySize`. | ||
| If this setting is not provided, it will be calculated by multiplying `spark.comet.memoryOverhead` by | ||
| `spark.comet.columnar.shuffle.memory.factor` (default value is `1.0`). If a shuffle exceeds this amount of memory | ||
| then the query will fail. | ||
| then the query will fail. For full details on configuring Comet memory in on-heap mode, see the [Advanced Memory Tuning] | ||
| section of this guide. | ||
|
|
||
| [shuffle]: #shuffle | ||
|
|
||
| [Advanced Memory Tuning]: #advanced-memory-tuning | ||
|
|
||
| ### Determining How Much Memory to Allocate | ||
|
|
||
| Generally, increasing the amount of memory allocated to Comet will improve query performance by reducing the | ||
|
|
@@ -102,36 +112,43 @@ Workarounds for this problem include: | |
|
|
||
| ## Advanced Memory Tuning | ||
|
|
||
| ### Configuring spark.executor.memoryOverhead in On-Heap Mode | ||
|
|
||
| In some environments, such as Kubernetes and YARN, it is important to correctly set `spark.executor.memoryOverhead` so | ||
| that it is possible to allocate off-heap memory when running in on-heap mode. | ||
|
|
||
| Comet will automatically set `spark.executor.memoryOverhead` based on the `spark.comet.memory*` settings so that | ||
| resource managers respect Apache Spark memory configuration before starting the containers. | ||
|
|
||
| ### Configuring Off-Heap Memory Pools | ||
|
|
||
| Comet implements multiple memory pool implementations. The type of pool can be specified with `spark.comet.exec.memoryPool`. | ||
|
|
||
| The valid pool types for off-heap mode are: | ||
|
|
||
| - `fair_unified` (default when `spark.memory.offHeap.enabled=true` is set) | ||
| - `fair_unified_global` (default when `spark.memory.offHeap.enabled=true` is set) | ||
| - `fair_unified` | ||
| - `greedy_unified` | ||
|
|
||
| Both of these pools share off-heap memory between Spark and Comet. This approach is referred to as | ||
| All of these pools share off-heap memory between Spark and Comet. This approach is referred to as | ||
| unified memory management. The size of the pool is specified by `spark.memory.offHeap.size`. | ||
|
|
||
| The `greedy_unified` pool type implements a greedy first-come first-serve limit. This pool works well for queries that do not | ||
| need to spill or have a single spillable operator. | ||
| Comet's memory accounting isn't 100% accurate and this can result in Comet using more memory than it reserves, | ||
| leading to out-of-memory exceptions. To work around this issue, it is possible to | ||
| set `spark.comet.exec.memoryPool.fraction` to a value less than `1.0` to restrict the amount of memory that can be | ||
| reserved by Comet. | ||
|
|
||
| The `fair_unified` pool type prevents operators from using more than an even fraction of the available memory | ||
| The `fair_unified` pool types prevents operators from using more than an even fraction of the available memory | ||
| (i.e. `pool_size / num_reservations`). This pool works best when you know beforehand | ||
| the query has multiple operators that will likely all need to spill. Sometimes it will cause spills even | ||
| when there is sufficient memory in order to leave enough memory for other operators. | ||
|
|
||
| `fair_unified_global` allows any task to use the full off-heap memory pool. | ||
|
||
|
|
||
| `fair_unified` restricts each task to using a fraction of the off-heap memory pool based on number of cores | ||
| and cores per task. | ||
|
|
||
| The `greedy_unified` pool type implements a greedy first-come first-serve limit. This pool works well for queries that do not | ||
| need to spill or have a single spillable operator. | ||
|
|
||
| ### Configuring On-Heap Memory Pools | ||
|
|
||
| ```{warning} | ||
| Support for on-heap memory pools is deprecated and will be removed from a future release. | ||
| ``` | ||
|
|
||
| When running in on-heap mode, Comet will use its own dedicated memory pools that are not shared with Spark. | ||
|
|
||
| The type of pool can be specified with `spark.comet.exec.memoryPool`. The default setting is `greedy_task_shared`. | ||
|
|
@@ -172,6 +189,14 @@ adjusting how much memory to allocate. | |
| [FairSpillPool]: https://docs.rs/datafusion/latest/datafusion/execution/memory_pool/struct.FairSpillPool.html | ||
| [UnboundedMemoryPool]: https://docs.rs/datafusion/latest/datafusion/execution/memory_pool/struct.UnboundedMemoryPool.html | ||
|
|
||
| ### Configuring spark.executor.memoryOverhead in On-Heap Mode | ||
|
|
||
| In some environments, such as Kubernetes and YARN, it is important to correctly set `spark.executor.memoryOverhead` so | ||
| that it is possible to allocate off-heap memory when running in on-heap mode. | ||
|
|
||
| Comet will automatically set `spark.executor.memoryOverhead` based on the `spark.comet.memory*` settings so that | ||
| resource managers respect Apache Spark memory configuration before starting the containers. | ||
|
|
||
| ## Optimizing Joins | ||
|
|
||
| Spark often chooses `SortMergeJoin` over `ShuffledHashJoin` for stability reasons. If the build-side of a | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -86,31 +86,13 @@ class CometExecIterator( | |
| val conf = SparkEnv.get.conf | ||
| val localDiskDirs = SparkEnv.get.blockManager.getLocalDiskDirs | ||
|
|
||
| val offHeapMode = CometSparkSessionExtensions.isOffHeapEnabled(conf) | ||
| val memoryLimit = if (offHeapMode) { | ||
| // in unified mode we share off-heap memory with Spark | ||
| ByteUnit.MiB.toBytes(conf.getSizeAsMb("spark.memory.offHeap.size")) | ||
| } else { | ||
| // we'll use the built-in memory pool from DF, and initializes with `memory_limit` | ||
| // and `memory_fraction` below. | ||
| CometSparkSessionExtensions.getCometMemoryOverhead(conf) | ||
| } | ||
|
|
||
| // serialize Comet related Spark configs in protobuf format | ||
| val builder = ConfigMap.newBuilder() | ||
| conf.getAll.filter(_._1.startsWith(CometConf.COMET_PREFIX)).foreach { case (k, v) => | ||
| builder.putEntries(k, v) | ||
| } | ||
| val protobufSparkConfigs = builder.build().toByteArray | ||
|
|
||
| val memoryLimitPerTask = if (offHeapMode) { | ||
| // this per-task limit is not used in native code when using unified memory | ||
| // so we can skip calculating it and avoid logging irrelevant information | ||
| 0 | ||
| } else { | ||
| getMemoryLimitPerTask(conf) | ||
| } | ||
|
|
||
| // Create keyUnwrapper if encryption is enabled | ||
| val keyUnwrapper = if (encryptedFilePaths.nonEmpty) { | ||
| val unwrapper = new CometFileKeyUnwrapper() | ||
|
|
@@ -124,6 +106,8 @@ class CometExecIterator( | |
| null | ||
| } | ||
|
|
||
| val memoryConfig = CometExecIterator.getMemoryConfig(conf) | ||
|
|
||
| nativeLib.createPlan( | ||
| id, | ||
| cometBatchIterators, | ||
|
|
@@ -135,10 +119,10 @@ class CometExecIterator( | |
| cometTaskMemoryManager, | ||
| localDiskDirs, | ||
| batchSize = COMET_BATCH_SIZE.get(), | ||
| offHeapMode, | ||
| memoryPoolType = COMET_EXEC_MEMORY_POOL_TYPE.get(), | ||
| memoryLimit, | ||
| memoryLimitPerTask, | ||
| memoryConfig.offHeapMode, | ||
| memoryConfig.memoryPoolType, | ||
| memoryConfig.memoryLimit, | ||
| memoryConfig.memoryLimitPerTask, | ||
| taskAttemptId, | ||
| keyUnwrapper) | ||
| } | ||
|
|
@@ -148,34 +132,6 @@ 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) | ||
| 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 | ||
| val limit = (maxMemory.toFloat * coresPerTask / numCores).toLong | ||
| logInfo( | ||
| s"Calculated per-task memory limit of $limit ($maxMemory * $coresPerTask / $numCores)") | ||
| limit | ||
| } | ||
|
|
||
| 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 | ||
| } | ||
| } | ||
|
|
||
| private def getNextBatch: Option[ColumnarBatch] = { | ||
| assert(partitionIndex >= 0 && partitionIndex < numParts) | ||
|
|
||
|
|
@@ -247,6 +203,8 @@ class CometExecIterator( | |
|
|
||
| nextBatch = getNextBatch | ||
|
|
||
| logTrace(s"Task $taskAttemptId memory pool usage is ${cometTaskMemoryManager.getUsed} bytes") | ||
|
|
||
| if (nextBatch.isEmpty) { | ||
| close() | ||
| false | ||
|
|
@@ -285,25 +243,11 @@ class CometExecIterator( | |
| traceMemoryUsage() | ||
| } | ||
|
|
||
| // The allocator thoughts the exported ArrowArray and ArrowSchema structs are not released, | ||
| // so it will report: | ||
| // Caused by: java.lang.IllegalStateException: Memory was leaked by query. | ||
| // Memory leaked: (516) Allocator(ROOT) 0/516/808/9223372036854775807 (res/actual/peak/limit) | ||
| // Suspect this seems a false positive leak, because there is no reported memory leak at JVM | ||
| // when profiling. `allocator` reports a leak because it calculates the accumulated number | ||
| // of memory allocated for ArrowArray and ArrowSchema. But these exported ones will be | ||
| // released in native side later. | ||
| // More to clarify it. For ArrowArray and ArrowSchema, Arrow will put a release field into the | ||
| // memory region which is a callback function pointer (C function) that could be called to | ||
| // release these structs in native code too. Once we wrap their memory addresses at native | ||
| // side using FFI ArrowArray and ArrowSchema, and drop them later, the callback function will | ||
| // be called to release the memory. | ||
| // But at JVM, the allocator doesn't know about this fact so it still keeps the accumulated | ||
| // number. | ||
| // Tried to manually do `release` and `close` that can make the allocator happy, but it will | ||
| // cause JVM runtime failure. | ||
|
|
||
| // allocator.close() | ||
|
Comment on lines
-288
to
-306
Member
Author
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 removed this comment since it refers to an allocator that no longer exists in this code. |
||
| val memInUse = cometTaskMemoryManager.getUsed | ||
| if (memInUse != 0) { | ||
| logWarning(s"CometExecIterator closed with non-zero memory usage : $memInUse") | ||
| } | ||
|
|
||
| closed = true | ||
| } | ||
| } | ||
|
|
@@ -318,3 +262,74 @@ class CometExecIterator( | |
| nativeLib.logMemoryUsage(s"task_memory_spark_$threadId", sparkTaskMemory) | ||
| } | ||
| } | ||
|
|
||
| object CometExecIterator extends Logging { | ||
|
|
||
| def getMemoryConfig(conf: SparkConf): MemoryConfig = { | ||
| val numCores = numDriverOrExecutorCores(conf).toFloat | ||
|
||
| val coresPerTask = conf.get("spark.task.cpus", "1").toFloat | ||
|
||
| // there are different paths for on-heap vs off-heap mode | ||
| val offHeapMode = CometSparkSessionExtensions.isOffHeapEnabled(conf) | ||
| if (offHeapMode) { | ||
| // in off-heap mode, Comet uses unified memory management to share off-heap memory with Spark | ||
| val offHeapSize = ByteUnit.MiB.toBytes(conf.getSizeAsMb("spark.memory.offHeap.size")) | ||
| val memoryFraction = CometConf.COMET_EXEC_MEMORY_POOL_FRACTION.get() | ||
| val memoryLimit = (offHeapSize * memoryFraction).toLong | ||
| val memoryLimitPerTask = (memoryLimit.toFloat * coresPerTask / numCores).toLong | ||
|
||
| val memoryPoolType = getMemoryPoolType(defaultValue = "fair_unified") | ||
| logInfo( | ||
| s"memoryPoolType=$memoryPoolType, " + | ||
| s"offHeapSize=${toMB(offHeapSize)}, " + | ||
| s"memoryFraction=$memoryFraction, " + | ||
| s"memoryLimit=${toMB(memoryLimit)}, " + | ||
| s"memoryLimitPerTask=${toMB(memoryLimitPerTask)}") | ||
| MemoryConfig(offHeapMode, memoryPoolType = memoryPoolType, memoryLimit, memoryLimitPerTask) | ||
| } else { | ||
| // we'll use the built-in memory pool from DF, and initializes with `memory_limit` | ||
| // and `memory_fraction` below. | ||
| val memoryLimit = CometSparkSessionExtensions.getCometMemoryOverhead(conf) | ||
| // example 16GB maxMemory * 16 cores with 4 cores per task results | ||
| // in memory_limit_per_task = 16 GB * 4 / 16 = 16 GB / 4 = 4GB | ||
| val memoryLimitPerTask = (memoryLimit.toFloat * coresPerTask / numCores).toLong | ||
|
||
| val memoryPoolType = getMemoryPoolType(defaultValue = "greedy_task_shared") | ||
| logInfo( | ||
| s"memoryPoolType=$memoryPoolType, " + | ||
| s"memoryLimit=${toMB(memoryLimit)}, " + | ||
| s"memoryLimitPerTask=${toMB(memoryLimitPerTask)}") | ||
| MemoryConfig(offHeapMode, memoryPoolType = memoryPoolType, memoryLimit, memoryLimitPerTask) | ||
| } | ||
| } | ||
|
|
||
| private def getMemoryPoolType(defaultValue: String): String = { | ||
| COMET_EXEC_MEMORY_POOL_TYPE.get() match { | ||
| case "default" => defaultValue | ||
| case other => other | ||
| } | ||
| } | ||
|
|
||
| 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 | ||
|
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. would be nice to comment what expression is looking for like
Member
Author
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 added some comments |
||
| 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 | ||
| } | ||
| } | ||
|
|
||
| private def toMB(n: Long): String = { | ||
| s"${(n.toDouble / 1024.0 / 1024.0).toLong} MB" | ||
| } | ||
| } | ||
|
|
||
| case class MemoryConfig( | ||
| offHeapMode: Boolean, | ||
| memoryPoolType: String, | ||
| memoryLimit: Long, | ||
| memoryLimitPerTask: Long) | ||
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.
Are we not limiting the available memory pools?
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.
I think we should, but as a separate PR
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.
oops .. this was actually a copy-paste error - these are the on-heap pools. Updated.