Skip to content

feat(java): add non-blocking AsyncScanner with CompletableFuture API#6102

Merged
LuQQiu merged 13 commits intolance-format:mainfrom
beinan:feat/java-async-scanner
Mar 26, 2026
Merged

feat(java): add non-blocking AsyncScanner with CompletableFuture API#6102
LuQQiu merged 13 commits intolance-format:mainfrom
beinan:feat/java-async-scanner

Conversation

@beinan
Copy link
Copy Markdown
Contributor

@beinan beinan commented Mar 5, 2026

Summary

Implements a parallel async scanner alongside the existing blocking LanceScanner to prevent thread starvation in Java query engines like Presto and Trino.

This PR adds AsyncScanner - a non-blocking alternative to LanceScanner that uses CompletableFuture for true async I/O operations.

Motivation

Query engines like Presto and Trino rely on non-blocking I/O to efficiently multiplex thousands of concurrent queries on a limited thread pool. The current LanceScanner blocks Java threads during Rust I/O operations, causing thread starvation and poor performance in these environments.

Key Features

Non-blocking I/O: Spawns Tokio tasks instead of blocking Java threads
CompletableFuture API: Native Java async patterns for seamless integration
Persistent JNI dispatcher: Single thread attached to JVM for zero-overhead callbacks
Task-based architecture: Uses task IDs instead of JNI refs to prevent memory leaks
Full feature parity: Supports filters, projections, vector search, FTS, aggregates
Clean cancellation: Proper task cleanup without resource leaks
Parallel to existing API: No breaking changes, LanceScanner unchanged

Architecture

The implementation uses the "Task ID + Dispatcher" pattern:

  1. Java manages futures: ConcurrentHashMap<taskId, CompletableFuture<Long>> maps task IDs to pending requests
  2. Rust spawns async tasks: Returns immediately while Tokio handles I/O in background
  3. Lock-free completion channel: Carries (taskId, resultPointer) from Tokio to dispatcher
  4. Persistent dispatcher thread: Attaches to JVM once, completes Java futures via cached JNI method IDs

Components

Rust (java/lance-jni/src/):

  • dispatcher.rs - Persistent JNI thread with cached method IDs for callbacks
  • task_tracker.rs - Thread-safe task registry using RwLock<HashMap>
  • async_scanner.rs - AsyncScanner with Tokio task spawning and JNI exports
  • lib.rs - Modified to add JNI_OnLoad hook for dispatcher initialization

Java (java/src/main/java/org/lance/ipc/):

  • AsyncScanner.java - CompletableFuture-based async API with task management

Tests (java/src/test/java/org/lance/):

  • AsyncScannerTest.java - 6 comprehensive examples demonstrating usage patterns

Usage Examples

Basic async scan

ScanOptions options = new ScanOptions.Builder().batchSize(20L).build();

try (AsyncScanner scanner = AsyncScanner.create(dataset, options, allocator)) {
    CompletableFuture<ArrowReader> future = scanner.scanBatchesAsync();
    ArrowReader reader = future.get(10, TimeUnit.SECONDS);
    
    while (reader.loadNextBatch()) {
        // Process batches without blocking
    }
}

Multiple concurrent scans (key benefit for Presto/Trino)

List<CompletableFuture<Integer>> futures = new ArrayList<>();

for (int i = 0; i < 100; i++) {
    AsyncScanner scanner = AsyncScanner.create(dataset, options, allocator);
    futures.add(scanner.scanBatchesAsync()
        .thenApply(reader -> processInBackground(reader)));
}

// All scans run in parallel without blocking threads!
CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
    .get(30, TimeUnit.SECONDS);

Testing

cd java
./mvnw test -Dtest=AsyncScannerTest
./mvnw compile  # Full build verification

All tests pass ✅

Compatibility

  • ✅ No breaking changes to existing APIs
  • LanceScanner remains unchanged
  • ✅ Uses same ScanOptions for consistency
  • ✅ Opt-in: users choose blocking or async based on their needs

Performance Benefits

For query engines running hundreds of concurrent queries:

  • Before: Thread pool exhaustion as threads block on I/O
  • After: Threads immediately return, I/O happens in background

Checklist

  • Rust code compiles (cargo check)
  • Java code compiles (./mvnw compile)
  • Code formatted (./mvnw spotless:apply && cargo fmt)
  • Comprehensive tests added (AsyncScannerTest.java)
  • Documentation in code examples
  • No breaking changes

Related Issues

Addresses the need for non-blocking I/O in Java query engines that integrate with LanceDB.

🤖 Generated with Claude Code

@github-actions github-actions Bot added enhancement New feature or request java labels Mar 5, 2026
@github-actions
Copy link
Copy Markdown
Contributor

github-actions Bot commented Mar 5, 2026

PR Review: feat(java): add non-blocking AsyncScanner with CompletableFuture API

P0: Race condition in start_scan — Java futures can hang forever

In async_scanner.rs, start_scan spawns the Tokio task before registering it in TASK_TRACKER:

// Task spawned first...
let handle = RT.spawn(async move {
    // ...scan work...
    if let Some(info) = TASK_TRACKER.complete(task_id).await {
        // send to dispatcher
    }
});

// ...registered second
RT.block_on(async {
    TASK_TRACKER.register(task_id, ...).await;
});

If the spawned task completes before register is called (entirely possible for small/cached datasets), TASK_TRACKER.complete(task_id) returns None, the dispatcher message is never sent, and the Java CompletableFuture hangs forever. Registration must happen before the task is spawned, or use a synchronization mechanism (e.g., a oneshot channel) to ensure ordering.

P1: Large-scale code duplication with blocking_scanner.rs

build_full_text_search_query (~130 lines) and the entire scanner-building logic in inner_create_async_scanner (~130 lines) are copy-pasted from blocking_scanner.rs. When either copy is updated (e.g., new scan options, FTS query types), the other will silently go stale. These should be extracted into shared helper functions.

P1: releaseNativeScanner JNI signature mismatch

Java declares:

private native void releaseNativeScanner(long handle);

But the Rust JNI function ignores the handle parameter entirely — it only takes (env, j_scanner) and uses env.take_rust_field(j_scanner, ...). The extra parameter is silently ignored due to C calling conventions, but the API is misleading and fragile. Either remove the unused handle parameter from Java, or align the signatures.

@beinan
Copy link
Copy Markdown
Contributor Author

beinan commented Mar 5, 2026

Review Comments Addressed

Thank you for the detailed review! I've addressed all three issues:

✅ P0: Fixed race condition in start_scan

Problem: Task could complete before registration, causing Java futures to hang forever.

Solution:

  • Clone the GlobalRef so the spawned task has its own copy
  • Always send completion message to dispatcher (not conditional on task tracker)
  • Task tracker is now only used for cancellation support

Code change:

// Clone ref for spawned task to prevent race condition
let global_ref_for_task = scanner_global_ref.clone();

let handle = RT.spawn(async move {
    // ... scan work ...
    
    // Always send message even if not tracked
    TASK_TRACKER.complete(task_id).await;
    dispatcher.send(DispatcherMessage {
        scanner_global_ref: global_ref_for_task,
        ...
    });
});

// Registration now only for cancellation
TASK_TRACKER.register(task_id, TaskInfo { ... }).await;

✅ P1: Removed code duplication with blocking_scanner.rs

Problem: build_full_text_search_query (~130 lines) was duplicated.

Solution:

  • Made function pub(crate) in blocking_scanner.rs
  • Import and reuse from async_scanner.rs
  • Removed 130+ lines of duplicated code
  • Both scanners now share same FTS query building logic

✅ P1: Fixed JNI signature mismatch

Problem: releaseNativeScanner had unused handle parameter in Java.

Solution:

  • Removed unused handle parameter from Java method signature
  • Now matches Rust implementation that only uses j_scanner
  • API is clearer and less fragile

All changes are in commit 951950fe. CI should pass now! 🚀

@beinan beinan force-pushed the feat/java-async-scanner branch from 77cd10f to 0d160ba Compare March 9, 2026 22:47
@LuQQiu
Copy link
Copy Markdown
Contributor

LuQQiu commented Mar 10, 2026

@hamersaw seems to be a performance improvement for presto/trino

Copy link
Copy Markdown
Contributor

@hamersaw hamersaw left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks! This looks really good, mostly just questions on correct cleanup / logging in case of failures.

// Attach ONCE and never detach - this is the key optimization
let mut env = jvm
.attach_current_thread_permanently()
.expect("Failed to attach dispatcher to JVM");
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We probably do not want to use expect anywhere in here because it will panic and crash the JVM. I think there are other places in the PR where we probably don't want to use expect as well.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good call, I will double check. But for the "expect" here, I just want the JVM crash to make it fail fast. Otherwise the issue will be very difficult to catch or debug later.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed in a06c692. Kept expect() here for fail-fast initialization (JVM should crash if dispatcher can't initialize), but replaced the runtime expect() at async_scanner.rs:91 with error logging to prevent crashes during normal operation.

Comment thread java/lance-jni/src/async_scanner.rs Outdated

// Send result to dispatcher for Java completion
let dispatcher = DISPATCHER.get().expect("Dispatcher not initialized");
let _ = dispatcher.send(DispatcherMessage {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lets capture and at least log the error here.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed in a06c692. Added error logging for both dispatcher initialization check and send failures.

Comment thread java/lance-jni/src/async_scanner.rs Outdated
Comment on lines +236 to +239
let scanner_guard =
unsafe { env.get_rust_field::<_, _, AsyncScanner>(&j_scanner, NATIVE_ASYNC_SCANNER)? };

scanner_guard.start_scan(task_id, scanner_global_ref);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
let scanner_guard =
unsafe { env.get_rust_field::<_, _, AsyncScanner>(&j_scanner, NATIVE_ASYNC_SCANNER)? };
scanner_guard.start_scan(task_id, scanner_global_ref);
// Clone the Arc<Scanner> and drop the MutexGuard before calling start_scan,
// which does block_on internally. Holding the guard across block_on risks deadlock.
let scanner = {
let guard =
unsafe { env.get_rust_field::<_, _, AsyncScanner>(&j_scanner, NATIVE_ASYNC_SCANNER)? };
guard.inner.clone()
};
AsyncScanner::start_scan(scanner, task_id, scanner_global_ref);

This way we ensure correct releasing of resources.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed in a06c692. Implemented exactly as suggested - clone the Arc<Scanner> and drop the guard before calling start_scan_with_scanner to prevent deadlock.

});

// Register task for cancellation support
RT.block_on(async {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need some kind of two-phase tracking here? For example, before we spawn the task ^^ we register it, and then here we commit it. In the current implementation, however unlikely, if the I/O task completes before we reach this statement it can drop the _cleanup_guard and things will never be cleaned up from the TASK_TRACKER. A two-phase commit would ensure this is impossible.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Excellent catch! Fixed in a06c692 with a three-step two-phase registration pattern:

  1. Pre-register with placeholder handle BEFORE spawning
  2. Spawn the actual task
  3. Update registration with real handle

This guarantees the task is tracked before the cleanup guard can run. Added TaskTracker::update_handle() method to atomically replace the placeholder with the real handle.

Copy link
Copy Markdown
Contributor Author

@beinan beinan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed in a06c692:

  • Kept expect() in dispatcher initialization for fail-fast behavior (as discussed)
  • Replaced runtime expect() at async_scanner.rs:91 with error logging to prevent JVM crashes during task completion
  • Added error logging for dispatcher.send() failures instead of silently ignoring them
  • Fixed deadlock by cloning Arc<Scanner> and dropping guard before calling start_scan_with_scanner
  • Implemented two-phase registration pattern: pre-register with placeholder → spawn task → update with real handle
  • Added TaskTracker::update_handle() for atomic handle updates

All issues have been addressed. The dispatcher initialization still fails fast as intended, but runtime operations now handle errors gracefully.

beinan and others added 12 commits March 18, 2026 02:34
Implements a parallel async scanner alongside the existing blocking
LanceScanner to prevent thread starvation in Java query engines like
Presto and Trino.

## Key Features

- **Non-blocking I/O**: Spawns Tokio tasks instead of blocking Java threads
- **CompletableFuture API**: Native Java async patterns for better integration
- **Persistent JNI dispatcher**: Single thread attached to JVM for zero-overhead callbacks
- **Task-based architecture**: Uses task IDs instead of JNI refs to prevent memory leaks
- **Full feature parity**: Supports filters, projections, vector search, FTS, aggregates
- **Clean cancellation**: Proper task cleanup without resource leaks

## Implementation

### Rust Components

- **dispatcher.rs**: Persistent JNI thread with cached method IDs for callbacks
- **task_tracker.rs**: Thread-safe task registry using RwLock<HashMap>
- **async_scanner.rs**: AsyncScanner with Tokio task spawning and JNI exports
- **lib.rs**: JNI_OnLoad hook to initialize global dispatcher on library load

### Java Components

- **AsyncScanner.java**: CompletableFuture-based async API with task management
- **AsyncScannerTest.java**: 6 comprehensive examples demonstrating usage patterns

## Architecture

Uses the "Task ID + Dispatcher" pattern:
1. Java manages futures in ConcurrentHashMap<taskId, CompletableFuture>
2. Rust spawns async I/O tasks and returns immediately
3. Lock-free channel carries completion messages
4. Persistent dispatcher thread completes Java futures via JNI callbacks

## Testing

```bash
./mvnw test -Dtest=AsyncScannerTest
```

## Compatibility

- Parallel to existing LanceScanner (no breaking changes)
- Same ScanOptions API for consistency
- Opt-in: users choose blocking or async based on needs

Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
- Add #[allow(clippy::too_many_arguments)] to inner_create_async_scanner
- Remove needless borrows in dispatcher.rs call_method_unchecked calls
P0: Fix race condition in start_scan
- Clone GlobalRef so spawned task has its own copy
- Always send completion message even if task wasn't tracked
- Prevents Java futures from hanging if task completes before registration

P1: Remove code duplication
- Extract build_full_text_search_query to pub(crate) in blocking_scanner
- Import and reuse from async_scanner instead of duplicating ~130 lines
- Ensures both scanners stay in sync when FTS query types are updated

P1: Fix JNI signature mismatch
- Remove unused 'handle' parameter from releaseNativeScanner in Java
- Aligns Java signature with Rust implementation that only uses j_scanner

Additional:
- Add #[allow(dead_code)] to scanner_global_ref field (used for cleanup)
- Suppress false positive warning about unused field
Improved code readability by extracting error and success handling
into separate helper functions, reducing nesting depth in the
dispatcher's event loop.

Changes:
- Extract handle_error() for error completion path
- Extract handle_success() for success completion path
- Use match expression instead of if-let for cleaner dispatch
Eliminated ~260 lines of duplicated scanner building logic between
blocking_scanner.rs and async_scanner.rs by introducing a shared
ScannerOptions struct and build_scanner_with_options function.

Changes:
- Add ScannerOptions struct to hold all JNI scanner parameters
- Add build_scanner_with_options() as shared builder function
- Refactor inner_create_scanner to use shared builder (~130 lines → ~45 lines)
- Refactor inner_create_async_scanner to use shared builder (~130 lines → ~45 lines)
- Remove unused imports from async_scanner.rs
- Add explicit lifetimes to all JObject parameters for type safety

Benefits:
- Single source of truth for scanner building logic
- Easier maintenance (fix bugs once, add features once)
- No performance penalty (struct is stack-allocated and inlined)
- Removes clippy "too many arguments" concerns from inner functions

All tests passing with zero performance regression.
Fix formatting issues caught by CI:
- Alphabetize imports (ScannerOptions before build_scanner_with_options)
- Align parameter comments consistently
- Format long function calls across multiple lines
- Simplify jvalue struct initialization

Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
Replace the "magic flag" pattern (error_msg: Option<String>) with
idiomatic Rust Result<i64, String> for better type safety and clarity.

Changes:
- DispatcherMessage.result: Result<i64, String> instead of separate
  result_ptr and error_msg fields
- Use Ok(ptr) for success case instead of (ptr, None) tuple
- Use Err(msg) for error case instead of (-1, Some(msg)) tuple
- Match on Result variants (Ok/Err) instead of Option (Some/None)

Benefits:
- Self-documenting: Result explicitly represents success/failure
- Type-safe: Can't accidentally use result_ptr when there's an error
- Idiomatic: Standard Rust pattern every developer understands
- No magic numbers: Eliminates the -1 sentinel value

All tests passing (AsyncScannerTest: 6/6).

Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
Implement TaskCleanupGuard using RAII to guarantee task removal from
HashMap, preventing memory leaks in two scenarios:
1. Race condition where task completes before registration
2. Task panic before manual cleanup

Changes:
- Add TaskCleanupGuard struct with Drop trait implementation
- Guard automatically removes task_id from tracker when dropped
- Works on normal completion, panic, or cancellation
- Use RT.spawn() in Drop instead of block_on() to avoid runtime nesting

Benefits:
- Compiler-guaranteed cleanup - impossible to forget
- Exception-safe - works even when task panics
- Zero runtime cost - compiler inlines the guard

TODO: Add timeout-based cleanup for additional defense-in-depth
(see detailed proposal in task_tracker.rs)

All tests passing (AsyncScannerTest: 6/6).

Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
AsyncScanner was missing the prefilter parameter that was added to
LanceScanner in upstream. This caused AsyncScanner to ignore the
prefilter setting from ScanOptions.

Changes:
- Add options.isPrefilter() call in createAsyncScanner invocation
- Add prefilter boolean parameter to native method signature

This brings AsyncScanner in sync with LanceScanner and ensures all
scanner options are properly forwarded.

Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
- Replace runtime expect() with error logging to prevent JVM crashes
- Log dispatcher send errors instead of silently ignoring
- Fix potential deadlock by cloning Arc<Scanner> before start_scan
- Implement two-phase registration to prevent race condition
- Add TaskTracker::update_handle() for atomic handle updates

Reviewer: hamersaw
Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
@beinan beinan force-pushed the feat/java-async-scanner branch from dc25fef to 4cebdbf Compare March 18, 2026 02:34
@beinan
Copy link
Copy Markdown
Contributor Author

beinan commented Mar 18, 2026

Hi @hamersaw, could you help take a look again? Thanks!

@beinan
Copy link
Copy Markdown
Contributor Author

beinan commented Mar 19, 2026

@hamersaw Circling back on this—since tomorrow is my last day at Uber, I’m aiming to wrap up all outstanding PRs today. Do you have time for a final look so we can get this merged?

result,
}) {
log::error!(
"Failed to send completion message for task {}: {}",
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we clean up the result ptr if any error happens during send?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch! Added cleanup for the FFI stream pointer in both error paths — when the dispatcher is not initialized and when send() fails. Also save the pointer before sending so we can reclaim it without cloning the result. (474a70c)


match msg.result {
Err(error) => {
handle_error(&mut env, scanner_obj, fail_method, msg.task_id, &error)
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what would happen if we have error when handling error? is there a way like finally to cleanup the things

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great question. Added env.exception_clear() after every failed JNI call (new_string, failTask, completeTask) to clear any pending exception and protect the dispatcher loop from corruption. For handle_success, we also free the FFI stream pointer since Java will never receive it. This effectively acts as a finally block for each message dispatch. (474a70c)

Address reviewer feedback on resource cleanup:
- async_scanner: free FFI stream pointer when dispatcher is missing or
  send fails, preventing memory leaks
- dispatcher: clear pending JNI exceptions after failed JNI calls to
  protect the dispatcher loop from corruption
- dispatcher: free FFI stream pointer when completeTask call fails,
  since Java will never receive it

Co-Authored-By: Claude Opus 4.6 (1M context) <noreply@anthropic.com>
@hamersaw
Copy link
Copy Markdown
Contributor

Code review

Found 2 issues:

  1. FFI stream pointer leaked when close() races with an in-flight task completing. When close() calls pendingTasks.clear(), if a Rust task has already completed and sent a DispatcherMessage with a resultPtr, the dispatcher thread calls completeTask(taskId, ptr) — but pendingTasks.get(taskId) returns null (map was cleared), so the FFI stream pointer is silently dropped without being freed. In handle_success (dispatcher.rs), the pointer is only freed when call_method_unchecked returns a JNI error, not when the Java method succeeds but does nothing with the pointer. Additionally, close() never calls future.completeExceptionally() on pending futures before clearing the map, so any caller holding a reference to those futures will hang indefinitely.

/** Called by Rust dispatcher thread via JNI to complete a task successfully. */
private void completeTask(long taskId, long resultPtr) {
CompletableFuture<Long> future = pendingTasks.get(taskId);
if (future != null) {
future.complete(resultPtr);
}
}
/** Called by Rust dispatcher thread via JNI to fail a task with an error. */
private void failTask(long taskId, String errorMessage) {
CompletableFuture<Long> future = pendingTasks.get(taskId);
if (future != null) {
future.completeExceptionally(new RuntimeException(errorMessage));
}
}

if (nativeAsyncScannerHandle != 0) {
// Cancel all pending tasks
for (Long taskId : pendingTasks.keySet()) {
nativeCancelTask(taskId);
}
pendingTasks.clear();
releaseNativeScanner();
nativeAsyncScannerHandle = 0;
}

if let Err(e) = result {
log::error!("Failed to call completeTask: {:?}", e);
// Clear any pending JNI exception to protect the dispatcher loop
let _ = env.exception_clear();
// Clean up the FFI stream since Java won't receive it
unsafe {
drop(Box::from_raw(
result_ptr as *mut arrow::ffi_stream::FFI_ArrowArrayStream,
));
}
log::debug!(
"Cleaned up FFI stream pointer for task {} after completeTask failure",
task_id
);
}

  1. testConcurrentAsyncScans creates AsyncScanner outside try-with-resources, risking native resource leak. The scanner is created at line 158 without try-with-resources. scanner.close() is only called inside a thenApply lambda (line 172) — if the lambda throws before reaching that line, the native scanner is never released. All other tests in the file correctly use try-with-resources. The blocking scanner's Rust code (blocking_scanner.rs lines 487-496) explicitly documents that try-with-resources is required to prevent native memory leaks.

AsyncScanner scanner = AsyncScanner.create(dataset, options, allocator);
// Chain async operations: scan -> read -> count rows -> cleanup
CompletableFuture<Integer> future =
scanner
.scanBatchesAsync()
.thenApply(
reader -> {
try {
int count = 0;
while (reader.loadNextBatch()) {
count += reader.getVectorSchemaRoot().getRowCount();
}
reader.close();
scanner.close();
return count;
} catch (Exception e) {
throw new RuntimeException(e);
}
});

🤖 Generated with Claude Code

- If this code review was useful, please react with 👍. Otherwise, react with 👎.

Copy link
Copy Markdown
Contributor

@hamersaw hamersaw left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Appreciate the effort on getting this right! It's going to be a big improvement I think.

@LuQQiu LuQQiu merged commit 7f6fd0e into lance-format:main Mar 26, 2026
13 checks passed
wjones127 pushed a commit to wjones127/lance that referenced this pull request Mar 29, 2026
…ance-format#6102)

## Summary

Implements a parallel async scanner alongside the existing blocking
`LanceScanner` to prevent thread starvation in Java query engines like
Presto and Trino.

This PR adds **AsyncScanner** - a non-blocking alternative to
`LanceScanner` that uses `CompletableFuture` for true async I/O
operations.

## Motivation

Query engines like Presto and Trino rely on non-blocking I/O to
efficiently multiplex thousands of concurrent queries on a limited
thread pool. The current `LanceScanner` blocks Java threads during Rust
I/O operations, causing thread starvation and poor performance in these
environments.

## Key Features

✅ **Non-blocking I/O**: Spawns Tokio tasks instead of blocking Java
threads
✅ **CompletableFuture API**: Native Java async patterns for seamless
integration
✅ **Persistent JNI dispatcher**: Single thread attached to JVM for
zero-overhead callbacks
✅ **Task-based architecture**: Uses task IDs instead of JNI refs to
prevent memory leaks
✅ **Full feature parity**: Supports filters, projections, vector search,
FTS, aggregates
✅ **Clean cancellation**: Proper task cleanup without resource leaks  
✅ **Parallel to existing API**: No breaking changes, `LanceScanner`
unchanged

## Architecture

The implementation uses the **"Task ID + Dispatcher" pattern**:

1. **Java manages futures**: `ConcurrentHashMap<taskId,
CompletableFuture<Long>>` maps task IDs to pending requests
2. **Rust spawns async tasks**: Returns immediately while Tokio handles
I/O in background
3. **Lock-free completion channel**: Carries `(taskId, resultPointer)`
from Tokio to dispatcher
4. **Persistent dispatcher thread**: Attaches to JVM once, completes
Java futures via cached JNI method IDs

### Components

**Rust (`java/lance-jni/src/`):**
- `dispatcher.rs` - Persistent JNI thread with cached method IDs for
callbacks
- `task_tracker.rs` - Thread-safe task registry using `RwLock<HashMap>`
- `async_scanner.rs` - AsyncScanner with Tokio task spawning and JNI
exports
- `lib.rs` - Modified to add `JNI_OnLoad` hook for dispatcher
initialization

**Java (`java/src/main/java/org/lance/ipc/`):**
- `AsyncScanner.java` - CompletableFuture-based async API with task
management

**Tests (`java/src/test/java/org/lance/`):**
- `AsyncScannerTest.java` - 6 comprehensive examples demonstrating usage
patterns

## Usage Examples

### Basic async scan
```java
ScanOptions options = new ScanOptions.Builder().batchSize(20L).build();

try (AsyncScanner scanner = AsyncScanner.create(dataset, options, allocator)) {
    CompletableFuture<ArrowReader> future = scanner.scanBatchesAsync();
    ArrowReader reader = future.get(10, TimeUnit.SECONDS);
    
    while (reader.loadNextBatch()) {
        // Process batches without blocking
    }
}
```

### Multiple concurrent scans (key benefit for Presto/Trino)
```java
List<CompletableFuture<Integer>> futures = new ArrayList<>();

for (int i = 0; i < 100; i++) {
    AsyncScanner scanner = AsyncScanner.create(dataset, options, allocator);
    futures.add(scanner.scanBatchesAsync()
        .thenApply(reader -> processInBackground(reader)));
}

// All scans run in parallel without blocking threads!
CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
    .get(30, TimeUnit.SECONDS);
```

## Testing

```bash
cd java
./mvnw test -Dtest=AsyncScannerTest
./mvnw compile  # Full build verification
```

All tests pass ✅

## Compatibility

- ✅ No breaking changes to existing APIs
- ✅ `LanceScanner` remains unchanged
- ✅ Uses same `ScanOptions` for consistency
- ✅ Opt-in: users choose blocking or async based on their needs

## Performance Benefits

For query engines running hundreds of concurrent queries:
- **Before**: Thread pool exhaustion as threads block on I/O
- **After**: Threads immediately return, I/O happens in background

## Checklist

- [x] Rust code compiles (`cargo check`)
- [x] Java code compiles (`./mvnw compile`)
- [x] Code formatted (`./mvnw spotless:apply && cargo fmt`)
- [x] Comprehensive tests added (`AsyncScannerTest.java`)
- [x] Documentation in code examples
- [x] No breaking changes

## Related Issues

Addresses the need for non-blocking I/O in Java query engines that
integrate with LanceDB.

🤖 Generated with [Claude Code](https://claude.com/claude-code)

---------

Co-authored-by: Claude Opus 4.6 <noreply@anthropic.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

enhancement New feature or request java

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants