-
Notifications
You must be signed in to change notification settings - Fork 167
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat: Introduce
CometTaskMemoryManager
and native side memory pool
- Loading branch information
Showing
15 changed files
with
270 additions
and
41 deletions.
There are no files selected for viewing
8 changes: 8 additions & 0 deletions
8
common/src/main/java/org/apache/comet/CometOutOfMemoryError.java
This file contains 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 |
---|---|---|
@@ -0,0 +1,8 @@ | ||
package org.apache.comet; | ||
|
||
/** OOM error specific for Comet memory management */ | ||
public class CometOutOfMemoryError extends OutOfMemoryError { | ||
public CometOutOfMemoryError(String msg) { | ||
super(msg); | ||
} | ||
} |
This file contains 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 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 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 |
---|---|---|
@@ -0,0 +1,85 @@ | ||
use std::{ | ||
fmt::{Debug, Formatter, Result as FmtResult}, | ||
sync::{ | ||
atomic::{AtomicUsize, Ordering::Relaxed}, | ||
Arc, | ||
}, | ||
}; | ||
|
||
use jni::objects::GlobalRef; | ||
|
||
use datafusion::{ | ||
common::DataFusionError, | ||
execution::memory_pool::{MemoryPool, MemoryReservation}, | ||
}; | ||
|
||
use crate::jvm_bridge::{jni_call, JVMClasses}; | ||
|
||
pub struct CometMemoryPool { | ||
task_memory_manager_handle: Arc<GlobalRef>, | ||
used: AtomicUsize, | ||
} | ||
|
||
impl Debug for CometMemoryPool { | ||
fn fmt(&self, f: &mut Formatter<'_>) -> FmtResult { | ||
f.debug_struct("CometMemoryPool") | ||
.field("used", &self.used.load(Relaxed)) | ||
.finish() | ||
} | ||
} | ||
|
||
impl CometMemoryPool { | ||
pub fn new(task_memory_manager_handle: Arc<GlobalRef>) -> CometMemoryPool { | ||
Self { | ||
task_memory_manager_handle, | ||
used: AtomicUsize::new(0), | ||
} | ||
} | ||
} | ||
|
||
unsafe impl Send for CometMemoryPool {} | ||
unsafe impl Sync for CometMemoryPool {} | ||
|
||
impl MemoryPool for CometMemoryPool { | ||
fn grow(&self, _: &MemoryReservation, additional: usize) { | ||
self.used.fetch_add(additional, Relaxed); | ||
} | ||
|
||
fn shrink(&self, _: &MemoryReservation, size: usize) { | ||
let mut env = JVMClasses::get_env(); | ||
let handle = self.task_memory_manager_handle.as_obj(); | ||
unsafe { | ||
jni_call!(&mut env, comet_task_memory_manager(handle).release_memory(size as i64) -> ()) | ||
.unwrap(); | ||
} | ||
self.used.fetch_sub(size, Relaxed); | ||
} | ||
|
||
fn try_grow(&self, _: &MemoryReservation, additional: usize) -> Result<(), DataFusionError> { | ||
if additional > 0 { | ||
let mut env = JVMClasses::get_env(); | ||
let handle = self.task_memory_manager_handle.as_obj(); | ||
unsafe { | ||
let acquired = jni_call!(&mut env, | ||
comet_task_memory_manager(handle).acquire_memory(additional as i64) -> i64)?; | ||
|
||
// If the number of bytes we acquired is less than the requested, return an error, | ||
// and hopefully will trigger spilling from the caller side. | ||
if acquired < additional as i64 { | ||
return Err(DataFusionError::Execution(format!( | ||
"Failed to acquire {} bytes, only got {}. Reserved: {}", | ||
additional, | ||
acquired, | ||
self.reserved(), | ||
))); | ||
} | ||
} | ||
self.used.fetch_add(additional, Relaxed); | ||
} | ||
Ok(()) | ||
} | ||
|
||
fn reserved(&self) -> usize { | ||
self.used.load(Relaxed) | ||
} | ||
} |
This file contains 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 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 |
---|---|---|
@@ -0,0 +1,45 @@ | ||
use jni::{ | ||
errors::Result as JniResult, | ||
objects::{JClass, JMethodID}, | ||
signature::{Primitive, ReturnType}, | ||
JNIEnv, | ||
}; | ||
|
||
use crate::jvm_bridge::get_global_jclass; | ||
|
||
/// A DataFusion `MemoryPool` implementation for Comet, which delegate to the JVM | ||
/// side `CometTaskMemoryManager`. | ||
#[derive(Debug)] | ||
pub struct CometTaskMemoryManager<'a> { | ||
pub class: JClass<'a>, | ||
pub method_acquire_memory: JMethodID, | ||
pub method_release_memory: JMethodID, | ||
|
||
pub method_acquire_memory_ret: ReturnType, | ||
pub method_release_memory_ret: ReturnType, | ||
} | ||
|
||
impl<'a> CometTaskMemoryManager<'a> { | ||
pub const JVM_CLASS: &'static str = "org/apache/spark/CometTaskMemoryManager"; | ||
|
||
pub fn new(env: &mut JNIEnv<'a>) -> JniResult<CometTaskMemoryManager<'a>> { | ||
let class = get_global_jclass(env, Self::JVM_CLASS)?; | ||
|
||
let result = CometTaskMemoryManager { | ||
class, | ||
method_acquire_memory: env.get_method_id( | ||
Self::JVM_CLASS, | ||
"acquireMemory", | ||
"(J)J".to_string(), | ||
)?, | ||
method_release_memory: env.get_method_id( | ||
Self::JVM_CLASS, | ||
"releaseMemory", | ||
"(J)V".to_string(), | ||
)?, | ||
method_acquire_memory_ret: ReturnType::Primitive(Primitive::Long), | ||
method_release_memory_ret: ReturnType::Primitive(Primitive::Void), | ||
}; | ||
Ok(result) | ||
} | ||
} |
This file contains 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 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
49 changes: 49 additions & 0 deletions
49
spark/src/main/java/org/apache/spark/CometTaskMemoryManager.java
This file contains 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 |
---|---|---|
@@ -0,0 +1,49 @@ | ||
package org.apache.spark; | ||
|
||
import java.io.IOException; | ||
|
||
import org.apache.spark.memory.MemoryConsumer; | ||
import org.apache.spark.memory.MemoryMode; | ||
import org.apache.spark.memory.TaskMemoryManager; | ||
|
||
/** | ||
* A adapter class that is used by Comet native to acquire & release memory through Spark's unified | ||
* memory manager. This assumes Spark's off-heap memory mode is enabled. | ||
*/ | ||
public class CometTaskMemoryManager { | ||
private final TaskMemoryManager internal; | ||
private final NativeMemoryConsumer nativeMemoryConsumer; | ||
|
||
public CometTaskMemoryManager() { | ||
this.internal = TaskContext$.MODULE$.get().taskMemoryManager(); | ||
this.nativeMemoryConsumer = new NativeMemoryConsumer(); | ||
} | ||
|
||
// Called by Comet native through JNI. | ||
// Returns the actual amount of memory (in bytes) granted. | ||
public long acquireMemory(long size) { | ||
return internal.acquireExecutionMemory(size, nativeMemoryConsumer); | ||
} | ||
|
||
// Called by Comet native through JNI | ||
public void releaseMemory(long size) { | ||
internal.releaseExecutionMemory(size, nativeMemoryConsumer); | ||
} | ||
|
||
/** | ||
* A dummy memory consumer that does nothing when spilling. At the moment, Comet native doesn't | ||
* share the same API as Spark and cannot trigger spill when acquire memory. Therefore, when | ||
* acquiring memory from native or JVM, spilling can only be triggered from JVM operators. | ||
*/ | ||
private class NativeMemoryConsumer extends MemoryConsumer { | ||
protected NativeMemoryConsumer() { | ||
super(CometTaskMemoryManager.this.internal, 0, MemoryMode.OFF_HEAP); | ||
} | ||
|
||
@Override | ||
public long spill(long size, MemoryConsumer trigger) throws IOException { | ||
// No spilling | ||
return 0; | ||
} | ||
} | ||
} |
This file contains 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 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
Oops, something went wrong.