This is an automated email from the ASF dual-hosted git repository.
mbutrovich pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/datafusion-comet.git
The following commit(s) were added to refs/heads/main by this push:
new 3a6452b3a perf: don't busy-poll Tokio stream for plans without
CometScan. (#3063)
3a6452b3a is described below
commit 3a6452b3aee2620146c029e5f156031c7c7937dc
Author: Matt Butrovich <[email protected]>
AuthorDate: Sat Jan 10 15:51:27 2026 -0500
perf: don't busy-poll Tokio stream for plans without CometScan. (#3063)
---
native/core/src/execution/jni_api.rs | 25 ++++++++++++++++---------
1 file changed, 16 insertions(+), 9 deletions(-)
diff --git a/native/core/src/execution/jni_api.rs
b/native/core/src/execution/jni_api.rs
index 41e1050b5..75c53198b 100644
--- a/native/core/src/execution/jni_api.rs
+++ b/native/core/src/execution/jni_api.rs
@@ -564,16 +564,23 @@ pub unsafe extern "system" fn
Java_org_apache_comet_Native_executePlan(
}
return Ok(-1);
}
- // A poll pending means there are more than one
blocking operators,
- // we don't need go back-forth between JVM/Native.
Just keeping polling.
+ // A poll pending means the stream is not ready yet.
Poll::Pending => {
- // TODO: Investigate if JNI calls are safe without
block_in_place.
- // block_in_place prevents Tokio from migrating
this task to another thread,
- // which is necessary because JNI env is
thread-local. If we can guarantee
- // thread safety another way, we could remove this
wrapper for better perf.
- tokio::task::block_in_place(|| {
- pull_input_batches(exec_context)
- })?;
+ if exec_context.scans.is_empty() {
+ // Pure async I/O (e.g., IcebergScanExec,
DataSourceExec)
+ // Yield to let the executor drive I/O instead
of busy-polling
+ tokio::task::yield_now().await;
+ } else {
+ // Has ScanExec operators
+ // Busy-poll to pull batches from JVM
+ // TODO: Investigate if JNI calls are safe
without block_in_place.
+ // block_in_place prevents Tokio from
migrating this task to another thread,
+ // which is necessary because JNI env is
thread-local. If we can guarantee
+ // thread safety another way, we could remove
this wrapper for better perf.
+ tokio::task::block_in_place(|| {
+ pull_input_batches(exec_context)
+ })?;
+ }
// Output not ready yet
continue;
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]