Skip to content

Commit e72beb1

Browse files
andygroveKontinuationviirya
authored
chore: Follow-on PR to fully enable onheap memory usage (#1210)
* Make datafusion's native memory pool configurable * save * fix * Update memory calculation and add draft documentation * ready for review * ready for review * address feedback * Update docs/source/user-guide/tuning.md Co-authored-by: Liang-Chi Hsieh <[email protected]> * Update docs/source/user-guide/tuning.md Co-authored-by: Kristin Cowalcijk <[email protected]> * Update docs/source/user-guide/tuning.md Co-authored-by: Liang-Chi Hsieh <[email protected]> * Update docs/source/user-guide/tuning.md Co-authored-by: Liang-Chi Hsieh <[email protected]> * remove unused config --------- Co-authored-by: Kristin Cowalcijk <[email protected]> Co-authored-by: Liang-Chi Hsieh <[email protected]>
1 parent 5c389d1 commit e72beb1

File tree

8 files changed

+64
-46
lines changed

8 files changed

+64
-46
lines changed

common/src/main/scala/org/apache/comet/CometConf.scala

Lines changed: 0 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -452,15 +452,6 @@ object CometConf extends ShimCometConf {
452452
.intConf
453453
.createWithDefault(8192)
454454

455-
val COMET_EXEC_MEMORY_FRACTION: ConfigEntry[Double] = conf("spark.comet.exec.memoryFraction")
456-
.doc(
457-
"The fraction of memory from Comet memory overhead that the native memory " +
458-
"manager can use for execution. The purpose of this config is to set aside memory for " +
459-
"untracked data structures, as well as imprecise size estimation during memory " +
460-
"acquisition.")
461-
.doubleConf
462-
.createWithDefault(0.7)
463-
464455
val COMET_PARQUET_ENABLE_DIRECT_BUFFER: ConfigEntry[Boolean] =
465456
conf("spark.comet.parquet.enable.directBuffer")
466457
.doc("Whether to use Java direct byte buffer when reading Parquet.")

docs/source/user-guide/configs.md

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,6 @@ Comet provides the following configuration settings.
4747
| spark.comet.exec.globalLimit.enabled | Whether to enable globalLimit by default. | true |
4848
| spark.comet.exec.hashJoin.enabled | Whether to enable hashJoin by default. | true |
4949
| spark.comet.exec.localLimit.enabled | Whether to enable localLimit by default. | true |
50-
| spark.comet.exec.memoryFraction | The fraction of memory from Comet memory overhead that the native memory manager can use for execution. The purpose of this config is to set aside memory for untracked data structures, as well as imprecise size estimation during memory acquisition. | 0.7 |
5150
| spark.comet.exec.memoryPool | The type of memory pool to be used for Comet native execution. Available memory pool types are 'greedy', 'fair_spill', 'greedy_task_shared', 'fair_spill_task_shared', 'greedy_global' and 'fair_spill_global', By default, this config is 'greedy_task_shared'. | greedy_task_shared |
5251
| spark.comet.exec.project.enabled | Whether to enable project by default. | true |
5352
| spark.comet.exec.replaceSortMergeJoin | Experimental feature to force Spark to replace SortMergeJoin with ShuffledHashJoin for improved performance. This feature is not stable yet. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | false |

docs/source/user-guide/tuning.md

Lines changed: 51 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -23,11 +23,52 @@ Comet provides some tuning options to help you get the best performance from you
2323

2424
## Memory Tuning
2525

26-
Comet shares an off-heap memory pool between Spark and Comet. This requires setting `spark.memory.offHeap.enabled=true`.
27-
If this setting is not enabled, Comet will not accelerate queries and will fall back to Spark.
26+
### Unified Memory Management with Off-Heap Memory
27+
28+
The recommended way to share memory between Spark and Comet is to set `spark.memory.offHeap.enabled=true`. This allows
29+
Comet to share an off-heap memory pool with Spark. 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.
30+
31+
### Dedicated Comet Memory Pools
32+
33+
Spark uses on-heap memory mode by default, i.e., the `spark.memory.offHeap.enabled` setting is not enabled. If Spark is under on-heap memory mode, Comet will use its own dedicated memory pools that
34+
are not shared with Spark. This requires additional configuration settings to be specified to set the size and type of
35+
memory pool to use.
36+
37+
The size of the pool can be set explicitly with `spark.comet.memoryOverhead`. If this setting is not specified then
38+
the memory overhead will be calculated by multiplying the executor memory by `spark.comet.memory.overhead.factor`
39+
(defaults to `0.2`).
40+
41+
The type of pool can be specified with `spark.comet.exec.memoryPool`. The default setting is `greedy_task_shared`.
42+
43+
The valid pool types are:
44+
45+
- `greedy`
46+
- `greedy_global`
47+
- `greedy_task_shared`
48+
- `fair_spill`
49+
- `fair_spill_global`
50+
- `fair_spill_task_shared`
51+
52+
Pool types ending with `_global` use a single global memory pool between all tasks on same executor.
53+
54+
Pool types ending with `_task_shared` share a single memory pool across all attempts for a single task.
55+
56+
Other pool types create a dedicated pool per native query plan using a fraction of the available pool size based on number of cores
57+
and cores per task.
58+
59+
The `greedy*` pool types use DataFusion's [GreedyMemoryPool], which implements a greedy first-come first-serve limit. This
60+
pool works well for queries that do not need to spill or have a single spillable operator.
61+
62+
The `fair_spill*` pool types use DataFusion's [FairSpillPool], which prevents spillable reservations from using more
63+
than an even fraction of the available memory sans any unspillable reservations
64+
(i.e. `(pool_size - unspillable_memory) / num_spillable_reservations)`). This pool works best when you know beforehand
65+
the query has multiple spillable operators that will likely all need to spill. Sometimes it will cause spills even
66+
when there was sufficient memory (reserved for other operators) to avoid doing so. Unspillable memory is allocated in
67+
a first-come, first-serve fashion
68+
69+
[GreedyMemoryPool]: https://docs.rs/datafusion/latest/datafusion/execution/memory_pool/struct.GreedyMemoryPool.html
70+
[FairSpillPool]: https://docs.rs/datafusion/latest/datafusion/execution/memory_pool/struct.FairSpillPool.html
2871

29-
Each executor will have a single memory pool which will be shared by all native plans being executed within that
30-
process, and by Spark itself. The size of the pool is specified by `spark.memory.offHeap.size`.
3172

3273
### Determining How Much Memory to Allocate
3374

@@ -106,15 +147,19 @@ then any shuffle operations that cannot be supported in this mode will fall back
106147
### Shuffle Compression
107148

108149
By default, Spark compresses shuffle files using LZ4 compression. Comet overrides this behavior with ZSTD compression.
109-
Compression can be disabled by setting `spark.shuffle.compress=false`, which may result in faster shuffle times in
150+
Compression can be disabled by setting `spark.shuffle.compress=false`, which may result in faster shuffle times in
110151
certain environments, such as single-node setups with fast NVMe drives, at the expense of increased disk space usage.
111152

112153
## Explain Plan
154+
113155
### Extended Explain
156+
114157
With Spark 4.0.0 and newer, Comet can provide extended explain plan information in the Spark UI. Currently this lists
115158
reasons why Comet may not have been enabled for specific operations.
116159
To enable this, in the Spark configuration, set the following:
160+
117161
```shell
118162
-c spark.sql.extendedExplainProviders=org.apache.comet.ExtendedExplainInfo
119163
```
120-
This will add a section to the detailed plan displayed in the Spark SQL UI page.
164+
165+
This will add a section to the detailed plan displayed in the Spark SQL UI page.

native/core/src/execution/jni_api.rs

Lines changed: 2 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -162,7 +162,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
162162
memory_pool_type: jstring,
163163
memory_limit: jlong,
164164
memory_limit_per_task: jlong,
165-
memory_fraction: jdouble,
166165
task_attempt_id: jlong,
167166
debug_native: jboolean,
168167
explain_native: jboolean,
@@ -208,7 +207,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_createPlan(
208207
memory_pool_type,
209208
memory_limit,
210209
memory_limit_per_task,
211-
memory_fraction,
212210
)?;
213211
let memory_pool =
214212
create_memory_pool(&memory_pool_config, task_memory_manager, task_attempt_id);
@@ -281,14 +279,13 @@ fn parse_memory_pool_config(
281279
memory_pool_type: String,
282280
memory_limit: i64,
283281
memory_limit_per_task: i64,
284-
memory_fraction: f64,
285282
) -> CometResult<MemoryPoolConfig> {
286283
let memory_pool_config = if use_unified_memory_manager {
287284
MemoryPoolConfig::new(MemoryPoolType::Unified, 0)
288285
} else {
289286
// Use the memory pool from DF
290-
let pool_size = (memory_limit as f64 * memory_fraction) as usize;
291-
let pool_size_per_task = (memory_limit_per_task as f64 * memory_fraction) as usize;
287+
let pool_size = memory_limit as usize;
288+
let pool_size_per_task = memory_limit_per_task as usize;
292289
match memory_pool_type.as_str() {
293290
"fair_spill_task_shared" => {
294291
MemoryPoolConfig::new(MemoryPoolType::FairSpillTaskShared, pool_size_per_task)

spark/src/main/java/org/apache/spark/shuffle/comet/CometShuffleMemoryAllocator.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,7 @@ public static CometShuffleMemoryAllocatorTrait getInstance(
5252
(boolean)
5353
CometConf$.MODULE$.COMET_COLUMNAR_SHUFFLE_UNIFIED_MEMORY_ALLOCATOR_IN_TEST().get();
5454

55-
if (isSparkTesting && !useUnifiedMemAllocator) {
55+
if (!useUnifiedMemAllocator) {
5656
synchronized (CometShuffleMemoryAllocator.class) {
5757
if (INSTANCE == null) {
5858
// CometTestShuffleMemoryAllocator handles pages by itself so it can be a singleton.

spark/src/main/scala/org/apache/comet/CometExecIterator.scala

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -20,10 +20,11 @@
2020
package org.apache.comet
2121

2222
import org.apache.spark._
23+
import org.apache.spark.internal.Logging
2324
import org.apache.spark.sql.comet.CometMetricNode
2425
import org.apache.spark.sql.vectorized._
2526

26-
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}
27+
import org.apache.comet.CometConf.{COMET_BATCH_SIZE, COMET_BLOCKING_THREADS, COMET_DEBUG_ENABLED, COMET_EXEC_MEMORY_POOL_TYPE, COMET_EXPLAIN_NATIVE_ENABLED, COMET_WORKER_THREADS}
2728
import org.apache.comet.vector.NativeUtil
2829

2930
/**
@@ -52,7 +53,8 @@ class CometExecIterator(
5253
nativeMetrics: CometMetricNode,
5354
numParts: Int,
5455
partitionIndex: Int)
55-
extends Iterator[ColumnarBatch] {
56+
extends Iterator[ColumnarBatch]
57+
with Logging {
5658

5759
private val nativeLib = new Native()
5860
private val nativeUtil = new NativeUtil()
@@ -75,7 +77,6 @@ class CometExecIterator(
7577
memory_pool_type = COMET_EXEC_MEMORY_POOL_TYPE.get(),
7678
memory_limit = CometSparkSessionExtensions.getCometMemoryOverhead(conf),
7779
memory_limit_per_task = getMemoryLimitPerTask(conf),
78-
memory_fraction = COMET_EXEC_MEMORY_FRACTION.get(),
7980
task_attempt_id = TaskContext.get().taskAttemptId,
8081
debug = COMET_DEBUG_ENABLED.get(),
8182
explain = COMET_EXPLAIN_NATIVE_ENABLED.get(),
@@ -94,7 +95,10 @@ class CometExecIterator(
9495
val coresPerTask = conf.get("spark.task.cpus", "1").toFloat
9596
// example 16GB maxMemory * 16 cores with 4 cores per task results
9697
// in memory_limit_per_task = 16 GB * 4 / 16 = 16 GB / 4 = 4GB
97-
(maxMemory.toFloat * coresPerTask / numCores).toLong
98+
val limit = (maxMemory.toFloat * coresPerTask / numCores).toLong
99+
logInfo(
100+
s"Calculated per-task memory limit of $limit ($maxMemory * $coresPerTask / $numCores)")
101+
limit
98102
}
99103

100104
private def numDriverOrExecutorCores(conf: SparkConf): Int = {

spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala

Lines changed: 2 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ import org.apache.spark.sql.types.{DoubleType, FloatType}
5353

5454
import org.apache.comet.CometConf._
5555
import org.apache.comet.CometExplainInfo.getActualPlan
56-
import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isOffHeapEnabled, isSpark34Plus, isSpark40Plus, isTesting, shouldApplySparkToColumnar, withInfo, withInfos}
56+
import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometNativeShuffleMode, isCometScan, isCometScanEnabled, isCometShuffleEnabled, isSpark34Plus, isSpark40Plus, shouldApplySparkToColumnar, withInfo, withInfos}
5757
import org.apache.comet.parquet.{CometParquetScan, SupportsComet}
5858
import org.apache.comet.rules.RewriteJoin
5959
import org.apache.comet.serde.OperatorOuterClass.Operator
@@ -919,14 +919,6 @@ class CometSparkSessionExtensions
919919
}
920920

921921
override def apply(plan: SparkPlan): SparkPlan = {
922-
923-
// Comet required off-heap memory to be enabled
924-
if (!isOffHeapEnabled(conf) && !isTesting) {
925-
logWarning("Comet native exec disabled because spark.memory.offHeap.enabled=false")
926-
withInfo(plan, "Comet native exec disabled because spark.memory.offHeap.enabled=false")
927-
return plan
928-
}
929-
930922
// DataFusion doesn't have ANSI mode. For now we just disable CometExec if ANSI mode is
931923
// enabled.
932924
if (isANSIEnabled(conf)) {
@@ -1187,21 +1179,12 @@ object CometSparkSessionExtensions extends Logging {
11871179
}
11881180
}
11891181

1190-
private[comet] def isOffHeapEnabled(conf: SQLConf): Boolean =
1191-
conf.getConfString("spark.memory.offHeap.enabled", "false").toBoolean
1192-
1193-
// Copied from org.apache.spark.util.Utils which is private to Spark.
1194-
private[comet] def isTesting: Boolean = {
1195-
System.getenv("SPARK_TESTING") != null || System.getProperty("spark.testing") != null
1196-
}
1197-
11981182
// Check whether Comet shuffle is enabled:
11991183
// 1. `COMET_EXEC_SHUFFLE_ENABLED` is true
12001184
// 2. `spark.shuffle.manager` is set to `CometShuffleManager`
12011185
// 3. Off-heap memory is enabled || Spark/Comet unit testing
12021186
private[comet] def isCometShuffleEnabled(conf: SQLConf): Boolean =
1203-
COMET_EXEC_SHUFFLE_ENABLED.get(conf) && isCometShuffleManagerEnabled(conf) &&
1204-
(isOffHeapEnabled(conf) || isTesting)
1187+
COMET_EXEC_SHUFFLE_ENABLED.get(conf) && isCometShuffleManagerEnabled(conf)
12051188

12061189
private[comet] def getCometShuffleNotEnabledReason(conf: SQLConf): Option[String] = {
12071190
if (!COMET_EXEC_SHUFFLE_ENABLED.get(conf)) {

spark/src/main/scala/org/apache/comet/Native.scala

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -55,7 +55,6 @@ class Native extends NativeBase {
5555
memory_pool_type: String,
5656
memory_limit: Long,
5757
memory_limit_per_task: Long,
58-
memory_fraction: Double,
5958
task_attempt_id: Long,
6059
debug: Boolean,
6160
explain: Boolean,

0 commit comments

Comments
 (0)