diff --git a/docs/LARGE_RESULT_SET_DEVELOPER_GUIDE.md b/docs/LARGE_RESULT_SET_DEVELOPER_GUIDE.md new file mode 100644 index 0000000..4e8c13a --- /dev/null +++ b/docs/LARGE_RESULT_SET_DEVELOPER_GUIDE.md @@ -0,0 +1,543 @@ +# Large Result Set Retrieval — Developer Guide + +Bulk retrieval for returning thousands to millions of rows from Quickwit +splits via Arrow FFI, with bounded memory. + +## Overview + +Two retrieval tiers are available: + +| Tier | Use Case | API | Memory | Latency | +|------|----------|-----|--------|---------| +| **Streaming** | Bulk unscored | `startStreamingRetrieval()` / `nextBatch()` | ~24MB fixed | Pipelined | +| **Scored** | Top-K ranked | `search()` + `docBatchArrowFfi()` | Proportional to K | Two calls | + +**Streaming works for all split types:** +- **Companion splits** (with parquet manifest) stream from parquet files via columnar reads +- **Regular splits** (tantivy doc store only) stream from the tantivy document store via `doc_async()` + +--- + +## Architecture + +``` +Java caller + │ + ├─ Streaming path ──────────────────────────────────────────┐ + │ startStreamingRetrieval() │ + │ → JNI: nativeStartStreamingRetrieval │ + │ 1. DocIdCollector search (no BM25) │ + │ ┌── companion split? ──────────────────────┐ │ + │ │ YES: resolve → parquet streaming │ │ + │ │ 2. Fast-field resolution to file+row │ │ + │ │ 3. start_streaming_retrieval() │ │ + │ │ (columnar parquet reads) │ │ + │ ├──────────────────────────────────────────┤ │ + │ │ NO: tantivy doc store streaming │ │ + │ │ 2. start_tantivy_streaming_retrieval() │ │ + │ │ (row-oriented doc_async reads) │ │ + │ └──────────────────────────────────────────┘ │ + │ → spawns tokio producer task │ + │ → returns session handle │ + │ │ + │ nextBatch() (called repeatedly) │ + │ → JNI: nativeNextBatch │ + │ → session.blocking_next() │ + │ → write_batch_to_ffi() │ + │ ← row count (>0 = data, 0 = end, -1 = error) │ + │ │ + │ session.close() (AutoCloseable) │ + │ → JNI: nativeCloseStreamingSession │ + │ → release_arc (ARC_REGISTRY) │ + └───────────────────────────────────────────────────────────┘ +``` + +--- + +## Rust Module Layout + +``` +native/src/ +├── split_searcher/ +│ ├── docid_collector.rs # Stage 1: No-score tantivy Collector +│ ├── bulk_retrieval.rs # Stage 1: Search + fast-field resolution +│ └── streaming_doc_retrieval.rs # Non-companion: tantivy doc store → Arrow +│ +└── parquet_companion/ + ├── streaming_ffi.rs # Session type + companion streaming pipeline + ├── read_strategy.rs # Adaptive I/O strategy (companion only) + └── arrow_ffi_export.rs # Shared: batch FFI export (pre-existing) +``` + +--- + +## Stage 1: DocIdCollector + Bulk Search + +### DocIdCollector (`docid_collector.rs`) + +A custom tantivy `Collector` that returns `Vec<(segment_ord, doc_id)>` pairs +without computing BM25 scores. + +```rust +pub struct DocIdCollector; + +// Key trait implementations: +impl Collector for DocIdCollector { + type Fruit = Vec<(u32, u32)>; + fn requires_scoring(&self) -> bool { false } // Skips term-frequency decompression +} + +impl SegmentCollector for DocIdSegmentCollector { + fn collect_block(&mut self, docs: &[DocId]) { ... } // Batch of up to 64 docs +} +``` + +**Why not Quickwit's `leaf_search_single_split`?** + +The standard Quickwit path allocates a `PartialHit` protobuf per document +(split_id string + sort values + score). For 2M results, that's 2M protobuf +allocations we don't need. `DocIdCollector` stores only 8 bytes per hit +(two `u32`s), and by returning `requires_scoring() = false`, tantivy uses +the batch `collect_block()` path (up to 64 docs at a time) and never +decompresses term frequencies. + +**Memory**: 8 bytes/hit. 2M hits = 16MB. + +### Bulk Retrieval (`bulk_retrieval.rs`) + +Three public functions used by the streaming path: + +```rust +/// No-score search: QueryAst JSON → Vec<(segment_ord, doc_id)> +pub fn perform_bulk_search( + ctx: &CachedSearcherContext, + query_ast_json: &str, +) -> Result> + +/// Resolve doc addresses → parquet file groups (entirely in Rust, no JNI) +pub async fn resolve_to_parquet_locations( + ctx: &Arc, + doc_ids: &[(u32, u32)], +) -> Result>> +// Returns: file_idx → [(original_index, row_in_file)], sorted by row_in_file + +/// Get parquet storage from context with diagnostic error messages +pub fn get_parquet_storage( + ctx: &CachedSearcherContext, +) -> Result> +``` + +**Resolution modes:** + +| Mode | Condition | Mechanism | +|------|-----------|-----------| +| **Fast-field** | `ctx.has_merge_safe_tracking == true` | O(1) lookup via `__pq_file_hash` / `__pq_row_in_file` columns | +| **Legacy** | `ctx.has_merge_safe_tracking == false` | Manifest-based positional resolution via `group_doc_addresses_by_file()` | + +--- + +## Stage 2: Streaming Pipeline + +### Session Model (`streaming_ffi.rs`) + +For result sets too large to materialize in one call, the streaming path +uses a producer/consumer model with bounded memory: + +``` + ┌─────────────┐ mpsc::channel(2) ┌──────────────┐ + │ Producer │ ──── RecordBatch ──────► │ Consumer │ + │ (tokio task)│ │ (JNI thread)│ + │ │ ~12MB per batch │ │ + │ Reads files │ max 2 in flight │ Writes FFI │ + │ sequentially│ = ~24MB peak │ per batch │ + └─────────────┘ └──────────────┘ +``` + +```rust +pub struct StreamingRetrievalSession { + receiver: mpsc::Receiver>, + schema: Arc, + _producer_handle: Option>, // None for empty sessions +} + +pub fn start_streaming_retrieval( + groups: HashMap>, + projected_fields: Option>, + manifest: Arc, + storage: Arc, + metadata_cache: Option, + byte_cache: Option, + coalesce_config: Option, +) -> Result +``` + +**Constants:** +```rust +const TARGET_BATCH_SIZE: usize = 128 * 1024; // 128K rows ≈ 12MB at ~100 bytes/row +``` + +### Producer Algorithm + +1. Sort file groups by `file_idx` for deterministic output order +2. For each file: + - Compute selectivity and select adaptive I/O strategy (see Stage 3) + - Read file's rows via `read_parquet_batches_for_file()` + - Feed batches through `BatchAccumulator` + - Rename columns (parquet → tantivy names) + - Normalize timestamps to microseconds (Spark compatibility) + - Send through channel (blocks if consumer is behind) +3. Flush remaining accumulated rows as final batch + +### BatchAccumulator + +Prevents emitting tiny batches from small files while avoiding unbounded memory: + +```rust +struct BatchAccumulator { + target_size: usize, // TARGET_BATCH_SIZE + pending: Vec, + pending_rows: usize, +} + +// push() returns vec![] until pending_rows >= target_size, +// then drains all pending into one concatenated batch. +// flush() returns any remaining rows at end of stream. +``` + +Example: 10 files each contributing 20K rows → accumulator concatenates +batches from ~7 files into one 128K-row output batch, then emits a second +batch with the remaining ~72K rows. + +### Java API — `StreamingSession` (AutoCloseable) + +`startStreamingRetrieval()` returns a `StreamingSession` wrapper that implements +`AutoCloseable`, providing double-close safety and synchronized access: + +```java +// Start session — returns AutoCloseable StreamingSession +try (SplitSearcher.StreamingSession session = + searcher.startStreamingRetrieval(queryJson, "field1", "field2")) { + + // Discover output schema width + int numCols = session.getColumnCount(); + + // Poll batches until stream ends + while (true) { + long[] arrayAddrs = new long[numCols]; // allocate FFI structs + long[] schemaAddrs = new long[numCols]; + // ... initialize FFI struct memory ... + + int rows = session.nextBatch(arrayAddrs, schemaAddrs); + if (rows == 0) break; // end of stream + if (rows < 0) throw new RuntimeException("streaming error"); + + // Import Arrow arrays from FFI addresses + // Process batch (e.g., pass to Spark ColumnarBatch) + } +} // session.close() called automatically — safe to call multiple times +``` + +**Return values for `nextBatch()`:** + +| Value | Meaning | +|-------|---------| +| `> 0` | Number of rows in this batch | +| `0` | End of stream — no more batches | +| `-1` | Error (Java exception thrown) | + +### Spark Integration Pattern + +```scala +// CompanionColumnarPartitionReader.scala +class CompanionColumnarPartitionReader extends PartitionReader[ColumnarBatch] { + private var session: SplitSearcher.StreamingSession = null + + override def next(): Boolean = { + val rows = session.nextBatch(arrayAddrs, schemaAddrs) + if (rows == 0) return false + importBatchFromFfi(rows) + true + } + + override def close(): Unit = { + if (session != null) session.close() // safe to call multiple times + } +} +``` + +--- + +## Stage 3: Adaptive I/O Strategy + +### ReadStrategy (`read_strategy.rs`) + +Selects the optimal parquet read approach per file based on selectivity +(fraction of rows needed). Minimizes S3 GET request count — the dominant +cost when data transfer is free (in-region). + +```rust +pub enum ReadStrategy { + PageLevel, // < 5% selectivity + CoalescedPageLevel, // 5-25% selectivity + FullColumnChunk, // 25-50% selectivity + FullRowGroup, // > 50% selectivity +} +``` + +**Decision diagram:** + +``` +Selectivity: 0% 5% 25% 50% 100% + ├──────────┼────────────┼─────────────┼─────────────┤ + │ PageLevel│ Coalesced │ FullColumn │ FullRowGroup│ + │ (surgical│ PageLevel │ Chunk │ (bulk read) │ + │ per-page│ (1MB gap) │ (4MB gap) │ (no row │ + │ reads) │ │ │ selection) │ + └──────────┴────────────┴─────────────┴─────────────┘ +``` + +**Coalesce configuration per strategy:** + +| Strategy | Max Gap | Max Total | Page Index | Row Selection | +|----------|---------|-----------|------------|---------------| +| PageLevel | 512KB (base) | 8MB (base) | Yes | Yes | +| CoalescedPageLevel | 1MB | 16MB | Yes | Yes | +| FullColumnChunk | 4MB | 32MB | No | Yes (post-decode) | +| FullRowGroup | 4MB | 64MB | No | No (filter via `take()`) | + +**Public API:** + +```rust +pub fn compute_selectivity(selected_rows: usize, total_rows: usize) -> f64; + +impl ReadStrategy { + pub fn for_selectivity(selectivity: f64) -> Self; + pub fn coalesce_config(&self, base: CoalesceConfig) -> CoalesceConfig; + pub fn use_row_selection(&self) -> bool; + pub fn use_page_index(&self) -> bool; +} + +// Per-row-group refinement for mixed-selectivity files +pub fn strategy_for_row_group(rows_in_rg: usize, rg_total_rows: usize) -> ReadStrategy; +``` + +**Why adaptive I/O matters:** + +At low selectivity (1% of rows), page-level reads avoid downloading 99% +of the data. But at high selectivity (80% of rows), page-level reads +generate many small S3 GETs that cost more in latency and request fees +than simply reading entire column chunks. The strategy selection +automatically transitions from surgical reads to bulk reads as +selectivity increases. + +**S3 request reduction by strategy** (5 columns, 10 row groups): + +| Strategy | Requests per File | When | +|----------|-------------------|------| +| PageLevel | 50-100 | < 5% selectivity | +| CoalescedPageLevel | 15-30 | 5-25% | +| FullColumnChunk | 5-10 | 25-50% | +| FullRowGroup | 1-3 | > 50% | + +### Integration + +The streaming producer automatically applies adaptive I/O per file: + +```rust +// In produce_batches(): +let selectivity = compute_selectivity(num_rows_in_file, file_entry.num_rows); +let strategy = ReadStrategy::for_selectivity(selectivity); +let effective_coalesce = strategy.coalesce_config(base); +// Pass effective_coalesce to read_parquet_batches_for_file() +``` + +--- + +## Performance Debugging + +All pipeline stages emit timing via `perf_println!()`, controlled by +the `TANTIVY4JAVA_DEBUG=1` environment variable: + +``` +⏱️ BULK_SEARCH: query parse + build took 2ms +⏱️ BULK_SEARCH: search returned 1500000 docs in 45ms (no scoring) +⏱️ BULK_RESOLVE: loaded __pq columns for 3 segments in 8ms +⏱️ BULK_RESOLVE: resolved 1500000 docs via fast fields in 12ms +⏱️ BULK_RESOLVE: grouped into 10 files in 3ms +⏱️ STREAMING: producer start — 10 files, 1500000 total docs +⏱️ STREAMING: file[0] selectivity=15.2% strategy=CoalescedPageLevel +⏱️ STREAMING: file[0] read 152000 rows in 4 batches, took 230ms +⏱️ STREAMING_JNI: nextBatch returned 131072 rows +⏱️ STREAMING: producer complete — 10 files, 1500000 rows emitted, took 1840ms +``` + +--- + +## Memory Budget + +| Component | Size | Lifetime | +|-----------|------|----------| +| DocIdCollector results | 8 bytes/hit | Until resolution completes | +| Resolution groups (HashMap) | ~24 bytes/hit | Until streaming starts | +| Streaming channel | 2 × ~12MB | Duration of session | +| Per-batch FFI export | ~12MB | Until Java imports batch | +| **Peak (streaming)** | **~50MB** | **Regardless of total rows** | + +For comparison, materializing 2M rows × 100 bytes with the old path would +require ~200MB of `PartialHit` protobufs plus ~200MB of document data. + +--- + +## Memory Safety + +### Session Handle Management (ARC_REGISTRY) + +Streaming session handles use the `ARC_REGISTRY` pattern instead of raw +`Box::into_raw`/`Box::from_raw`. This provides: + +- **No use-after-free**: Handles are registry IDs, not raw pointers. A stale + handle returns `None` instead of dereferencing freed memory. +- **Safe double-close**: `release_arc()` is idempotent — calling it twice on + the same handle is a no-op (the registry entry is already removed). +- **Thread safety**: The registry uses `RwLock` for concurrent access. + +```rust +// Creating a session: +let handle = arc_to_jlong(Arc::new(Mutex::new(session))); + +// Using a session: +let arc = jlong_to_arc::>(handle) + .ok_or_else(|| anyhow!("Session not found or already closed"))?; + +// Closing a session (idempotent): +release_arc(handle); +``` + +### Arrow FFI Buffer Lifecycle + +`write_batch_to_ffi()` drops previous FFI contents before writing new ones, +preventing Arrow buffer leaks when FFI addresses are reused across batches: + +```rust +unsafe { + // Drop previous FFI contents to release Arrow buffers + let prev_array = std::ptr::read_unaligned(array_ptr); + drop(prev_array); + let prev_schema = std::ptr::read_unaligned(schema_ptr); + drop(prev_schema); + // Write new contents + std::ptr::write_unaligned(array_ptr, FFI_ArrowArray::new(&data)); + std::ptr::write_unaligned(schema_ptr, FFI_ArrowSchema::try_from(&field)?); +} +``` + +### `StreamingSession` Java Wrapper + +The `StreamingSession` class wraps the raw native handle with: + +- **`synchronized` methods**: Prevents concurrent `nextBatch`/`close` races +- **`volatile boolean closed`**: Double-close protection without synchronization overhead on reads +- **`AutoCloseable`**: Works with try-with-resources for automatic cleanup + +--- + +## Fast Field Warmup + +### Selective Warmup in Bulk Search + +`perform_bulk_search()` warms fast fields **only for fields that need them** — +specifically range queries and field_presence/exists queries. Term, phrase, and +wildcard queries do NOT trigger fast field warmup. + +```rust +// Extract only fields involved in range or field_presence queries +let fast_field_names = extract_range_query_fields(query_ast_json); + +// Warm only those specific fields (not all fast fields) +if !fast_field_names.is_empty() { + let field_names: Vec = fast_field_names.into_iter().collect(); + let _ = ctx.warm_native_fast_fields_l1_for_fields(&field_names).await; +} +``` + +**Why selective?** Fast field warmup triggers async reads from storage (S3/Azure) +into the HotDirectory cache. Warming all fast fields is expensive and wastes +bandwidth. The `extract_range_query_fields()` function (from `field_extraction.rs`) +parses the query AST JSON for `"type": "range"` and `"type": "field_presence"` +nodes, returning only the field names that actually need fast field access. + +| Query Type | Needs Fast Field Warmup? | Reason | +|------------|--------------------------|--------| +| Term | No | Uses term dictionary + postings | +| Phrase | No | Uses term dictionary + positions | +| Wildcard | No | Uses term dictionary FST | +| Range | **Yes** | Reads fast field column data | +| Exists/FieldPresence | **Yes** | Checks fast field presence | + +--- + +## Error Handling + +| Error | Cause | Recovery | +|-------|-------|----------| +| `startStreamingRetrieval` throws `IllegalStateException` | Not companion, or query parse error | Check split type / query JSON | +| `session.nextBatch()` returns `-1` | Parquet read failure, storage error | Check exception message, retry | +| `session.nextBatch()` returns `0` | Normal end of stream | Stop polling | +| `session.nextBatch()` after `close()` | Session already closed | Throws `IllegalStateException` | +| `session.close()` called twice | Already closed | No-op (safe) | +| Producer error mid-stream | S3 timeout, corrupt parquet | Error surfaced via next `nextBatch()` call | + +The streaming producer communicates errors through the channel — if a file +read fails, the error is sent as `Err(...)` on the channel and surfaces +as a `-1` return (with Java exception) on the next `nextBatch()` call. + +The `StreamingSession` wrapper is `AutoCloseable` and safe to call `close()` +multiple times. If the session is dropped without calling `close()` (e.g., +exception during processing), the native `Arc` in the registry will be cleaned +up when the registry entry is eventually released. The producer detects the +closed channel on its next `tx.send()` and stops gracefully. + +--- + +## Test Coverage + +| Module | Tests | What's covered | +|--------|-------|----------------| +| `docid_collector` | 4 | Collector trait, batch collection, merge_fruits, scoring disabled | +| `streaming_ffi` | 6 | BatchAccumulator push/flush/empty, timestamp normalization, type mapping | +| `read_strategy` | 7 | Selectivity thresholds, boundary values, coalesce scaling, flags | +| **Rust Total** | **17** | All unit-testable logic | + +| Java Test Class | Tests | What's covered | +|-----------------|-------|----------------| +| `LargeResultSetRetrievalTest` | ~20 | Error handling, query types (term/boolean/range/exists/wildcard), edge cases (empty results, single row), concurrency, field projection, cross-path consistency, scale, StreamingSession lifecycle | +| `ParquetCompanionTest` | 22 | End-to-end companion mode with real splits | +| `ParquetCompanionAggregationTest` | 43 | Aggregations on companion splits | + +--- + +## Adding New Features + +### Adding a new ReadStrategy tier + +1. Add variant to `ReadStrategy` enum in `read_strategy.rs` +2. Update `for_selectivity()` thresholds +3. Set `coalesce_config()`, `use_row_selection()`, `use_page_index()` for the new tier +4. Add tests for the new boundary values + +### Supporting per-row-group strategy selection + +The `strategy_for_row_group()` function is implemented but not yet wired +into the read pipeline. To enable it: + +1. In `read_parquet_batches_for_file()`, compute selectivity per row group + instead of per file +2. Apply different coalesce configs per row group fetch +3. For `FullRowGroup` row groups, skip `RowSelection` construction + +### Adding new projected field types + +Column type mapping is in `streaming_ffi::parquet_type_to_arrow_type()`. +Add new `tantivy_type` or `parquet_type` matches there. The actual data +types come from parquet file metadata at read time — this mapping is only +for schema construction. diff --git a/native/src/parquet_companion/arrow_ffi_export.rs b/native/src/parquet_companion/arrow_ffi_export.rs index 0f649f5..e74ead2 100644 --- a/native/src/parquet_companion/arrow_ffi_export.rs +++ b/native/src/parquet_companion/arrow_ffi_export.rs @@ -266,7 +266,7 @@ fn is_identity_permutation(perm: &[usize]) -> bool { /// /// This is a zero-copy operation — only the schema metadata changes, the data /// buffers are shared via Arc. -fn rename_columns_to_tantivy( +pub(crate) fn rename_columns_to_tantivy( batch: &RecordBatch, column_mapping: &[ColumnMapping], ) -> Result { diff --git a/native/src/parquet_companion/mod.rs b/native/src/parquet_companion/mod.rs index cdb8bab..5fc8a04 100644 --- a/native/src/parquet_companion/mod.rs +++ b/native/src/parquet_companion/mod.rs @@ -27,6 +27,8 @@ pub mod hash_field_rewriter; pub mod hash_touchup; pub mod string_indexing; pub(crate) mod page_index; +pub mod streaming_ffi; +pub mod read_strategy; pub use manifest::{ ParquetManifest, FastFieldMode, SegmentRowRange, ParquetFileEntry, diff --git a/native/src/parquet_companion/read_strategy.rs b/native/src/parquet_companion/read_strategy.rs new file mode 100644 index 0000000..23dbc07 --- /dev/null +++ b/native/src/parquet_companion/read_strategy.rs @@ -0,0 +1,236 @@ +// read_strategy.rs - Adaptive I/O strategy for parquet companion reads +// +// Selects the optimal read strategy per-file (or per-row-group) based on +// selectivity — the fraction of rows needed from the file. Low selectivity +// warrants surgical page-level reads; high selectivity is better served by +// reading full column chunks or entire row groups to minimize S3 request count. +// +// In-region S3 cost model: $0.0004/GET, $0.00/GB transfer. +// The only cost is request count, so gap-filling is almost always worthwhile. + +use super::cached_reader::CoalesceConfig; + +/// Per-file I/O strategy determined by selectivity. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum ReadStrategy { + /// < 5% selectivity: surgical page-level reads with RowSelection. + /// Uses page index offsets for byte-level precision. + /// Coalesce: base config (default 512KB gap). + PageLevel, + + /// 5-25% selectivity: still use RowSelection to skip unneeded rows, + /// but with larger coalesce gap to merge nearby page reads. + /// Coalesce: 1MB gap, 16MB max. + CoalescedPageLevel, + + /// 25-50% selectivity: read full column chunks per selected row group. + /// Skip page index — read the whole column in one request per RG. + /// Still apply RowSelection for row-level filtering after decode. + /// Coalesce: 4MB gap, 32MB max. + FullColumnChunk, + + /// > 50% selectivity: read entire selected row groups without + /// RowSelection. Let the downstream `take()` filter rows. + /// Maximum I/O efficiency — fewest S3 requests possible. + /// Coalesce: 4MB gap, 64MB max. + FullRowGroup, +} + +impl ReadStrategy { + /// Select read strategy based on selectivity (fraction of rows needed). + pub fn for_selectivity(selectivity: f64) -> Self { + if selectivity > 0.50 { + ReadStrategy::FullRowGroup + } else if selectivity > 0.25 { + ReadStrategy::FullColumnChunk + } else if selectivity > 0.05 { + ReadStrategy::CoalescedPageLevel + } else { + ReadStrategy::PageLevel + } + } + + /// Compute the effective CoalesceConfig for this strategy. + /// Uses the base config for PageLevel, progressively larger gaps for others. + pub fn coalesce_config(&self, base: CoalesceConfig) -> CoalesceConfig { + match self { + ReadStrategy::PageLevel => base, + ReadStrategy::CoalescedPageLevel => CoalesceConfig { + max_gap: 1024 * 1024, // 1MB + max_total: 16 * 1024 * 1024, // 16MB + }, + ReadStrategy::FullColumnChunk => CoalesceConfig { + max_gap: 4 * 1024 * 1024, // 4MB + max_total: 32 * 1024 * 1024, // 32MB + }, + ReadStrategy::FullRowGroup => CoalesceConfig { + max_gap: 4 * 1024 * 1024, // 4MB + max_total: 64 * 1024 * 1024, // 64MB + }, + } + } + + /// Whether to use RowSelection for row-level filtering. + /// FullRowGroup reads everything and filters via take() later. + pub fn use_row_selection(&self) -> bool { + match self { + ReadStrategy::PageLevel + | ReadStrategy::CoalescedPageLevel + | ReadStrategy::FullColumnChunk => true, + ReadStrategy::FullRowGroup => false, + } + } + + /// Whether to use page index for byte-level read precision. + /// Only needed for surgical page-level reads. + pub fn use_page_index(&self) -> bool { + match self { + ReadStrategy::PageLevel | ReadStrategy::CoalescedPageLevel => true, + ReadStrategy::FullColumnChunk | ReadStrategy::FullRowGroup => false, + } + } +} + +/// Compute selectivity: fraction of file rows selected. +pub fn compute_selectivity(selected_rows: usize, total_rows: usize) -> f64 { + if total_rows == 0 { + return 1.0; + } + selected_rows as f64 / total_rows as f64 +} + +/// Per-row-group strategy selection for mixed-selectivity files. +/// +/// A file with 10% overall selectivity might have 90% in one row group +/// and 0% in the rest. The row-group filter handles the 0% case (skips +/// entirely). For the 90% row group, we use FullRowGroup even though +/// the file average is only 10%. +pub fn strategy_for_row_group(rows_in_rg: usize, rg_total_rows: usize) -> ReadStrategy { + let rg_selectivity = compute_selectivity(rows_in_rg, rg_total_rows); + ReadStrategy::for_selectivity(rg_selectivity) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_selectivity_thresholds() { + assert_eq!(ReadStrategy::for_selectivity(0.01), ReadStrategy::PageLevel); + assert_eq!(ReadStrategy::for_selectivity(0.04), ReadStrategy::PageLevel); + assert_eq!( + ReadStrategy::for_selectivity(0.06), + ReadStrategy::CoalescedPageLevel + ); + assert_eq!( + ReadStrategy::for_selectivity(0.24), + ReadStrategy::CoalescedPageLevel + ); + assert_eq!( + ReadStrategy::for_selectivity(0.30), + ReadStrategy::FullColumnChunk + ); + assert_eq!( + ReadStrategy::for_selectivity(0.49), + ReadStrategy::FullColumnChunk + ); + assert_eq!( + ReadStrategy::for_selectivity(0.51), + ReadStrategy::FullRowGroup + ); + assert_eq!( + ReadStrategy::for_selectivity(1.0), + ReadStrategy::FullRowGroup + ); + } + + #[test] + fn test_boundary_values() { + // Exact boundary: 0.05 is PageLevel (not >0.05) + assert_eq!(ReadStrategy::for_selectivity(0.05), ReadStrategy::PageLevel); + // Just above + assert_eq!( + ReadStrategy::for_selectivity(0.0501), + ReadStrategy::CoalescedPageLevel + ); + // 0.25 boundary + assert_eq!( + ReadStrategy::for_selectivity(0.25), + ReadStrategy::CoalescedPageLevel + ); + assert_eq!( + ReadStrategy::for_selectivity(0.2501), + ReadStrategy::FullColumnChunk + ); + // 0.50 boundary + assert_eq!( + ReadStrategy::for_selectivity(0.50), + ReadStrategy::FullColumnChunk + ); + assert_eq!( + ReadStrategy::for_selectivity(0.5001), + ReadStrategy::FullRowGroup + ); + } + + #[test] + fn test_coalesce_config_scaling() { + let base = CoalesceConfig { + max_gap: 512 * 1024, + max_total: 8 * 1024 * 1024, + }; + + let page = ReadStrategy::PageLevel.coalesce_config(base); + assert_eq!(page.max_gap, 512 * 1024); + + let coalesced = ReadStrategy::CoalescedPageLevel.coalesce_config(base); + assert_eq!(coalesced.max_gap, 1024 * 1024); + + let full_col = ReadStrategy::FullColumnChunk.coalesce_config(base); + assert_eq!(full_col.max_gap, 4 * 1024 * 1024); + + let full_rg = ReadStrategy::FullRowGroup.coalesce_config(base); + assert_eq!(full_rg.max_gap, 4 * 1024 * 1024); + assert_eq!(full_rg.max_total, 64 * 1024 * 1024); + } + + #[test] + fn test_use_row_selection() { + assert!(ReadStrategy::PageLevel.use_row_selection()); + assert!(ReadStrategy::CoalescedPageLevel.use_row_selection()); + assert!(ReadStrategy::FullColumnChunk.use_row_selection()); + assert!(!ReadStrategy::FullRowGroup.use_row_selection()); + } + + #[test] + fn test_use_page_index() { + assert!(ReadStrategy::PageLevel.use_page_index()); + assert!(ReadStrategy::CoalescedPageLevel.use_page_index()); + assert!(!ReadStrategy::FullColumnChunk.use_page_index()); + assert!(!ReadStrategy::FullRowGroup.use_page_index()); + } + + #[test] + fn test_compute_selectivity() { + assert_eq!(compute_selectivity(0, 1000), 0.0); + assert_eq!(compute_selectivity(100, 1000), 0.1); + assert_eq!(compute_selectivity(500, 1000), 0.5); + assert_eq!(compute_selectivity(1000, 1000), 1.0); + // Edge case: empty file + assert_eq!(compute_selectivity(0, 0), 1.0); + } + + #[test] + fn test_strategy_for_row_group() { + // A row group where we need 90% of rows + assert_eq!( + strategy_for_row_group(900, 1000), + ReadStrategy::FullRowGroup + ); + // A row group where we need 1% of rows + assert_eq!( + strategy_for_row_group(10, 1000), + ReadStrategy::PageLevel + ); + } +} diff --git a/native/src/parquet_companion/streaming_ffi.rs b/native/src/parquet_companion/streaming_ffi.rs new file mode 100644 index 0000000..2d96da2 --- /dev/null +++ b/native/src/parquet_companion/streaming_ffi.rs @@ -0,0 +1,658 @@ +// streaming_ffi.rs - Streaming Arrow FFI retrieval for large companion-mode result sets +// +// Implements a session-based streaming pipeline for returning millions of rows +// from companion-mode splits without materializing everything in memory. +// +// Architecture: +// start_streaming_retrieval() → spawns tokio producer task +// producer reads files sequentially, accumulates TARGET_BATCH_SIZE batches +// sends batches through mpsc::channel(2) → consumer calls next_batch() +// +// Memory bound: ~24MB peak (2 × 12MB batches in channel) regardless of total rows. +// +// This is the single companion retrieval path for all result sizes. + +use std::collections::HashMap; +use std::sync::Arc; + +use anyhow::{Context, Result}; +use arrow::ffi::{FFI_ArrowArray, FFI_ArrowSchema}; +use arrow_array::RecordBatch; +use arrow_schema::{DataType, Field, Schema, TimeUnit}; +use quickwit_storage::Storage; +use tokio::sync::mpsc; + +use crate::perf_println; +use super::arrow_ffi_export::rename_columns_to_tantivy; +use super::arrow_to_tant::read_parquet_batches_for_file; +use super::cached_reader::{ByteRangeCache, CoalesceConfig}; +use super::manifest::{ColumnMapping, ParquetManifest}; +use super::read_strategy::{ReadStrategy, compute_selectivity}; +use super::transcode::MetadataCache; + +/// Target rows per output batch. Balances Spark vectorized execution +/// efficiency with memory footprint. +/// At ~100 bytes/row average: 128K rows ≈ 12MB per batch. +const TARGET_BATCH_SIZE: usize = 128 * 1024; + +/// Opaque handle for a streaming retrieval session. +/// Holds the consumer end of the channel and the output schema. +pub struct StreamingRetrievalSession { + receiver: mpsc::Receiver>, + schema: Arc, + _producer_handle: Option>, +} + +impl StreamingRetrievalSession { + /// Create a new streaming session with an active producer. + pub(crate) fn new( + receiver: mpsc::Receiver>, + schema: Arc, + producer_handle: tokio::task::JoinHandle<()>, + ) -> Self { + Self { + receiver, + schema, + _producer_handle: Some(producer_handle), + } + } + + /// Create an empty session for queries with no matches. + /// The receiver channel is already closed (sender dropped), so blocking_next + /// will immediately return None. + pub fn new_empty( + receiver: mpsc::Receiver>, + schema: Arc, + ) -> Self { + Self { + receiver, + schema, + _producer_handle: None, + } + } + + /// Get the output schema (with tantivy field names). + pub fn schema(&self) -> &Arc { + &self.schema + } + + /// Get the number of columns in the output. + pub fn num_columns(&self) -> usize { + self.schema.fields().len() + } + + /// Receive the next batch from the producer. + /// Returns None when the stream is complete. + pub fn blocking_next(&mut self) -> Option> { + self.receiver.blocking_recv() + } +} + +/// Start a streaming retrieval session. +/// +/// Returns immediately with a session handle. The producer runs on the +/// tokio runtime and feeds batches through channel(2) — it works one +/// batch ahead while Java processes the current batch. +/// +/// # Arguments +/// * `groups` - File groups: file_idx → [(original_index, row_in_file)] +/// * `projected_fields` - Optional field name projection +/// * `manifest` - Parquet companion manifest +/// * `storage` - Storage backend for reading parquet files +/// * `metadata_cache` - Optional parquet metadata cache +/// * `byte_cache` - Optional byte range cache +/// * `coalesce_config` - Optional coalescing configuration +pub fn start_streaming_retrieval( + groups: HashMap>, + projected_fields: Option>, + manifest: Arc, + storage: Arc, + metadata_cache: Option, + byte_cache: Option, + coalesce_config: Option, +) -> Result { + // Determine output schema from manifest column mapping + let tantivy_schema = build_tantivy_schema(&manifest, projected_fields.as_deref())?; + + let (tx, rx) = mpsc::channel::>(2); + + let projected_fields_arc: Option> = + projected_fields.map(|f| f.into()); + + let handle = tokio::spawn(async move { + let tx_err = tx.clone(); + let result = produce_batches( + tx, + groups, + projected_fields_arc, + &manifest, + &storage, + metadata_cache.as_ref(), + byte_cache.as_ref(), + coalesce_config, + ) + .await; + + if let Err(e) = result { + let _ = tx_err.send(Err(e)).await; + } + }); + + Ok(StreamingRetrievalSession::new(rx, tantivy_schema, handle)) +} + +/// Producer: reads files sequentially, emits TARGET_BATCH_SIZE batches. +/// +/// Files are processed in file_idx order for deterministic output. +/// Rows within each file are already sorted by row_in_file (from Phase 2). +/// The BatchAccumulator combines small file results into larger batches. +async fn produce_batches( + tx: mpsc::Sender>, + groups: HashMap>, + projected_fields: Option>, + manifest: &ParquetManifest, + storage: &Arc, + metadata_cache: Option<&MetadataCache>, + byte_cache: Option<&ByteRangeCache>, + coalesce_config: Option, +) -> Result<()> { + let t_total = std::time::Instant::now(); + let total_docs: usize = groups.values().map(|v| v.len()).sum(); + + perf_println!( + "⏱️ STREAMING: producer start — {} files, {} total docs", + groups.len(), total_docs + ); + + // Sort file groups by file_idx for deterministic output order + let mut sorted_groups: Vec<(usize, Vec<(usize, u64)>)> = groups.into_iter().collect(); + sorted_groups.sort_by_key(|(file_idx, _)| *file_idx); + + let mut accumulator = BatchAccumulator::new(TARGET_BATCH_SIZE); + let mut files_processed = 0usize; + let mut rows_emitted = 0usize; + + for (file_idx, rows) in sorted_groups { + let t_file = std::time::Instant::now(); + let num_rows_in_file = rows.len(); + + // Adaptive I/O strategy: select coalesce config based on per-file selectivity + if file_idx >= manifest.parquet_files.len() { + return Err(anyhow::anyhow!( + "Invalid file_idx {} >= {} parquet files in manifest", + file_idx, manifest.parquet_files.len() + )); + } + let file_entry = &manifest.parquet_files[file_idx]; + let selectivity = compute_selectivity(num_rows_in_file, file_entry.num_rows as usize); + let strategy = ReadStrategy::for_selectivity(selectivity); + let effective_coalesce = coalesce_config + .map(|base| strategy.coalesce_config(base)) + .or_else(|| { + // When no base config provided, use strategy defaults with a reasonable base + let base = CoalesceConfig { + max_gap: 512 * 1024, + max_total: 8 * 1024 * 1024, + }; + Some(strategy.coalesce_config(base)) + }); + + perf_println!( + "⏱️ STREAMING: file[{}] selectivity={:.1}% strategy={:?}", + file_idx, selectivity * 100.0, strategy + ); + + // Read this file's rows as RecordBatches + let batches = read_parquet_batches_for_file( + file_idx, + &rows, + projected_fields.as_deref().map(|s| s as &[String]), + manifest, + storage, + metadata_cache, + byte_cache, + effective_coalesce, + ) + .await?; + + perf_println!( + "⏱️ STREAMING: file[{}] read {} rows in {} batches, took {}ms", + file_idx, num_rows_in_file, batches.len(), t_file.elapsed().as_millis() + ); + + // Feed batches through the accumulator → emit TARGET_BATCH_SIZE chunks + for batch in batches { + let emitted = accumulator.push(batch)?; + for emit_batch in emitted { + let renamed = rename_columns_to_tantivy(&emit_batch, &manifest.column_mapping)?; + let normalized = normalize_timestamps(renamed)?; + rows_emitted += normalized.num_rows(); + if tx.send(Ok(normalized)).await.is_err() { + if rows_emitted == 0 { + return Err(anyhow::anyhow!( + "Consumer dropped before any data was sent — possible consumer error" + )); + } + perf_println!( + "⏱️ STREAMING: consumer dropped after {} rows — stopping producer", + rows_emitted + ); + return Ok(()); + } + } + } + + files_processed += 1; + } + + // Flush remaining rows + if let Some(final_batch) = accumulator.flush()? { + let renamed = rename_columns_to_tantivy(&final_batch, &manifest.column_mapping)?; + let normalized = normalize_timestamps(renamed)?; + rows_emitted += normalized.num_rows(); + let _ = tx.send(Ok(normalized)).await; + } + + perf_println!( + "⏱️ STREAMING: producer complete — {} files, {} rows emitted, took {}ms", + files_processed, rows_emitted, t_total.elapsed().as_millis() + ); + + Ok(()) +} + +/// Accumulates RecordBatches from file reads into TARGET_BATCH_SIZE output batches. +/// +/// Prevents emitting tiny batches (bad for Spark vectorized execution) and avoids +/// materializing everything at once (unbounded memory). +struct BatchAccumulator { + target_size: usize, + pending: Vec, + pending_rows: usize, +} + +impl BatchAccumulator { + fn new(target_size: usize) -> Self { + Self { + target_size, + pending: Vec::new(), + pending_rows: 0, + } + } + + /// Push a batch. Returns any batches that should be emitted (0 or 1). + fn push(&mut self, batch: RecordBatch) -> Result> { + let rows = batch.num_rows(); + self.pending.push(batch); + self.pending_rows += rows; + + if self.pending_rows >= self.target_size { + Ok(vec![self.drain()?]) + } else { + Ok(vec![]) + } + } + + /// Concatenate pending batches into one, reset accumulator. + fn drain(&mut self) -> Result { + let batches = std::mem::take(&mut self.pending); + self.pending_rows = 0; + if batches.len() == 1 { + Ok(batches.into_iter().next().unwrap()) + } else { + let schema = batches[0].schema(); + arrow::compute::concat_batches(&schema, batches.iter()) + .context("BatchAccumulator: failed to concat pending batches") + } + } + + /// Flush any remaining pending batches. + fn flush(self) -> Result> { + if self.pending.is_empty() { + Ok(None) + } else if self.pending.len() == 1 { + Ok(self.pending.into_iter().next()) + } else { + let schema = self.pending[0].schema(); + Ok(Some( + arrow::compute::concat_batches(&schema, self.pending.iter()) + .context("BatchAccumulator: failed to flush pending batches")?, + )) + } + } +} + +/// Normalize timestamps to microseconds for Spark compatibility. +/// Spark only supports Timestamp(MICROSECOND). +fn normalize_timestamps(batch: RecordBatch) -> Result { + let schema = batch.schema(); + let mut needs_cast = false; + + // Check if any timestamp columns need casting + for field in schema.fields() { + if let DataType::Timestamp(unit, _) = field.data_type() { + if *unit != TimeUnit::Microsecond { + needs_cast = true; + break; + } + } + } + + if !needs_cast { + return Ok(batch); + } + + // Cast timestamp columns to microseconds + let new_fields: Vec> = schema + .fields() + .iter() + .map(|f| match f.data_type() { + DataType::Timestamp(unit, tz) if *unit != TimeUnit::Microsecond => { + Arc::new(f.as_ref().clone().with_data_type( + DataType::Timestamp(TimeUnit::Microsecond, tz.clone()), + )) + } + _ => f.clone(), + }) + .collect(); + + let new_columns: Vec<_> = batch + .columns() + .iter() + .zip(schema.fields().iter()) + .map(|(col, field)| match field.data_type() { + DataType::Timestamp(unit, tz) if *unit != TimeUnit::Microsecond => { + let target = DataType::Timestamp(TimeUnit::Microsecond, tz.clone()); + arrow::compute::cast(col.as_ref(), &target) + .context("Failed to cast timestamp to microseconds") + } + _ => Ok(col.clone()), + }) + .collect::>>()?; + + let new_schema = Arc::new(Schema::new(new_fields)); + RecordBatch::try_new(new_schema, new_columns) + .context("Failed to create timestamp-normalized RecordBatch") +} + +/// Public entry point for building the tantivy schema from a manifest. +/// Used by JNI code to create schema for empty sessions. +pub fn build_tantivy_schema_pub( + manifest: &ParquetManifest, + projected_fields: Option<&[String]>, +) -> Result> { + build_tantivy_schema(manifest, projected_fields) +} + +/// Build the output schema with tantivy field names from manifest column mapping. +/// Optionally filtered by projected_fields. +fn build_tantivy_schema( + manifest: &ParquetManifest, + projected_fields: Option<&[String]>, +) -> Result> { + let mappings: Vec<&ColumnMapping> = if let Some(fields) = projected_fields { + manifest + .column_mapping + .iter() + .filter(|m| fields.iter().any(|f| f == &m.tantivy_field_name)) + .collect() + } else { + manifest.column_mapping.iter().collect() + }; + + if mappings.is_empty() { + anyhow::bail!("No matching columns found in manifest for streaming schema"); + } + + let fields: Vec = mappings + .iter() + .map(|m| { + let dt = parquet_type_to_arrow_type(&m.parquet_type, &m.tantivy_type); + Field::new(&m.tantivy_field_name, dt, true) + }) + .collect(); + + Ok(Arc::new(Schema::new(fields))) +} + +/// Map manifest parquet_type string to Arrow DataType. +/// This is a best-effort mapping for schema construction; the actual data types +/// come from the parquet file metadata during reads. +fn parquet_type_to_arrow_type(parquet_type: &str, tantivy_type: &str) -> DataType { + match tantivy_type { + "IpAddr" => DataType::Utf8, + "Bool" => DataType::Boolean, + "F64" => DataType::Float64, + "I64" => DataType::Int64, + "U64" => DataType::UInt64, + "Date" => DataType::Timestamp(TimeUnit::Microsecond, None), + "Bytes" => DataType::Binary, + _ => match parquet_type { + "BOOLEAN" => DataType::Boolean, + "INT32" => DataType::Int32, + "INT64" => DataType::Int64, + "FLOAT" => DataType::Float32, + "DOUBLE" => DataType::Float64, + "BYTE_ARRAY" | "FIXED_LEN_BYTE_ARRAY" => DataType::Utf8, + _ => DataType::Utf8, + }, + } +} + +/// Write a RecordBatch to pre-allocated FFI addresses. +/// Same pattern as arrow_ffi_export.rs but operates on a single batch. +/// +/// IMPORTANT: If the FFI addresses have been written to before (e.g., in a +/// streaming loop), the previous `FFI_ArrowArray`/`FFI_ArrowSchema` values are +/// read and dropped first to release their Arrow buffer memory via the `release` +/// callback. Callers that allocate fresh zeroed memory for each call are also safe +/// (a zeroed `FFI_ArrowArray` has `release = null`, which means no-op on drop). +/// +/// Timestamps are normalized to microseconds for Spark compatibility. The +/// streaming path's `produce_batches` already normalizes, so the check here +/// is a fast no-op in that case. +pub(crate) fn write_batch_to_ffi( + batch: &RecordBatch, + array_addrs: &[i64], + schema_addrs: &[i64], +) -> Result { + let num_cols = batch.num_columns(); + if array_addrs.len() < num_cols || schema_addrs.len() < num_cols { + anyhow::bail!( + "Insufficient FFI addresses: need {} columns but got {} array_addrs and {} schema_addrs", + num_cols, + array_addrs.len(), + schema_addrs.len() + ); + } + + let schema = batch.schema(); + for (i, col) in batch.columns().iter().enumerate() { + if array_addrs[i] == 0 || schema_addrs[i] == 0 { + anyhow::bail!( + "Null FFI address for column {}: array_addr={}, schema_addr={}", + i, + array_addrs[i], + schema_addrs[i] + ); + } + + // Normalize timestamps to microseconds (fast no-op if already microsecond) + let cast_col; + let col_for_export: &dyn arrow_array::Array = match col.data_type() { + DataType::Timestamp(unit, tz) if *unit != TimeUnit::Microsecond => { + let target = DataType::Timestamp(TimeUnit::Microsecond, tz.clone()); + cast_col = arrow::compute::cast(col.as_ref(), &target) + .context(format!( + "Failed to cast column {} from {:?} to Timestamp(Microsecond)", + i, + col.data_type() + ))?; + cast_col.as_ref() + } + _ => col.as_ref(), + }; + + // Handle non-zero offset + let data = if col_for_export.offset() != 0 { + let take_indices = + arrow_array::UInt32Array::from_iter_values(0..batch.num_rows() as u32); + arrow::compute::take(col_for_export, &take_indices, None) + .context("Failed to normalize column offset via take()")? + .to_data() + } else { + col_for_export.to_data() + }; + + let array_ptr = array_addrs[i] as *mut FFI_ArrowArray; + let schema_ptr = schema_addrs[i] as *mut FFI_ArrowSchema; + + let orig_field = schema.field(i); + let export_field: Field = match orig_field.data_type() { + DataType::Timestamp(unit, tz) if *unit != TimeUnit::Microsecond => orig_field + .as_ref() + .clone() + .with_data_type(DataType::Timestamp(TimeUnit::Microsecond, tz.clone())), + _ => orig_field.as_ref().clone(), + }; + + unsafe { + // Drop previous FFI contents to release Arrow buffers. + // A zeroed struct (release = null) is safe to read+drop (no-op). + let prev_array = std::ptr::read_unaligned(array_ptr); + drop(prev_array); + let prev_schema = std::ptr::read_unaligned(schema_ptr); + drop(prev_schema); + + std::ptr::write_unaligned(array_ptr, FFI_ArrowArray::new(&data)); + std::ptr::write_unaligned( + schema_ptr, + FFI_ArrowSchema::try_from(&export_field).map_err(|e| { + anyhow::anyhow!( + "FFI_ArrowSchema conversion failed for column {}: {}", + i, + e + ) + })?, + ); + } + } + + Ok(batch.num_rows()) +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow_array::*; + + #[test] + fn test_batch_accumulator_basic() { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let mut acc = BatchAccumulator::new(100); + + // Push small batch (50 rows) — should not emit + let batch1 = RecordBatch::try_new( + schema.clone(), + vec![Arc::new(Int32Array::from_iter_values(0..50))], + ) + .unwrap(); + let emitted = acc.push(batch1).unwrap(); + assert!(emitted.is_empty()); + + // Push another small batch (60 rows, total 110 >= 100) — should emit + let batch2 = RecordBatch::try_new( + schema.clone(), + vec![Arc::new(Int32Array::from_iter_values(50..110))], + ) + .unwrap(); + let emitted = acc.push(batch2).unwrap(); + assert_eq!(emitted.len(), 1); + assert_eq!(emitted[0].num_rows(), 110); + } + + #[test] + fn test_batch_accumulator_flush() { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let mut acc = BatchAccumulator::new(1000); + + // Push small batch — won't emit + let batch = RecordBatch::try_new( + schema.clone(), + vec![Arc::new(Int32Array::from_iter_values(0..10))], + ) + .unwrap(); + let emitted = acc.push(batch).unwrap(); + assert!(emitted.is_empty()); + + // Flush should return the pending batch + let flushed = acc.flush().unwrap(); + assert!(flushed.is_some()); + assert_eq!(flushed.unwrap().num_rows(), 10); + } + + #[test] + fn test_batch_accumulator_empty_flush() { + let acc = BatchAccumulator::new(100); + let flushed = acc.flush().unwrap(); + assert!(flushed.is_none()); + } + + #[test] + fn test_normalize_timestamps_noop() { + // No timestamp columns → should be a noop + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)])); + let batch = RecordBatch::try_new( + schema, + vec![Arc::new(Int64Array::from(vec![1, 2, 3]))], + ) + .unwrap(); + let result = normalize_timestamps(batch.clone()).unwrap(); + assert_eq!(result.schema(), batch.schema()); + } + + #[test] + fn test_normalize_timestamps_cast() { + // Nanosecond timestamps should be cast to microseconds + let schema = Arc::new(Schema::new(vec![Field::new( + "ts", + DataType::Timestamp(TimeUnit::Nanosecond, None), + false, + )])); + let batch = RecordBatch::try_new( + schema, + vec![Arc::new( + TimestampNanosecondArray::from(vec![1_000_000_000i64, 2_000_000_000i64]), + )], + ) + .unwrap(); + let result = normalize_timestamps(batch).unwrap(); + assert_eq!( + *result.schema().field(0).data_type(), + DataType::Timestamp(TimeUnit::Microsecond, None) + ); + let col = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(col.value(0), 1_000_000); // ns → µs + assert_eq!(col.value(1), 2_000_000); + } + + #[test] + fn test_parquet_type_to_arrow_type() { + assert_eq!(parquet_type_to_arrow_type("INT64", "I64"), DataType::Int64); + assert_eq!(parquet_type_to_arrow_type("BOOLEAN", "Bool"), DataType::Boolean); + assert_eq!(parquet_type_to_arrow_type("DOUBLE", "F64"), DataType::Float64); + assert_eq!(parquet_type_to_arrow_type("BYTE_ARRAY", "Str"), DataType::Utf8); + assert_eq!( + parquet_type_to_arrow_type("BYTE_ARRAY", "IpAddr"), + DataType::Utf8 + ); + assert_eq!( + parquet_type_to_arrow_type("INT64", "Date"), + DataType::Timestamp(TimeUnit::Microsecond, None) + ); + } +} diff --git a/native/src/split_searcher/bulk_retrieval.rs b/native/src/split_searcher/bulk_retrieval.rs new file mode 100644 index 0000000..3725282 --- /dev/null +++ b/native/src/split_searcher/bulk_retrieval.rs @@ -0,0 +1,329 @@ +// bulk_retrieval.rs - Bulk search and parquet location resolution +// +// Provides the shared foundation for both fused (Phase 0) and streaming (Phase 3) +// companion-mode retrieval: +// +// 1. perform_bulk_search() — No-score search via DocIdCollector, bypassing +// Quickwit's leaf_search path entirely. Returns Vec<(segment_ord, doc_id)>. +// +// 2. resolve_to_parquet_locations() — Resolves doc addresses to parquet file +// locations entirely in Rust via __pq_file_hash / __pq_row_in_file fast fields. +// No JNI round-trip needed. + +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; + +use anyhow::{anyhow, Context, Result}; + +use quickwit_query::query_ast::QueryAst; +use quickwit_query::create_default_quickwit_tokenizer_manager; + +use crate::perf_println; +use super::types::CachedSearcherContext; +use super::docid_collector::DocIdCollector; + +/// Execute a no-score search using DocIdCollector. +/// +/// Parses the Quickwit QueryAst JSON and converts it to a tantivy Query, +/// then executes directly against the cached tantivy Searcher with our +/// custom DocIdCollector (no BM25 scoring, batch collection). +/// +/// Before searching, warms up term dictionaries and posting lists for all +/// indexed fields so that the synchronous search can access them through +/// the HotDirectory cache (which only supports async reads for cold data). +/// +/// This bypasses Quickwit's leaf_search_single_split entirely — no PartialHit +/// protobuf allocation, no split_id strings, no sort values. +/// +/// # Arguments +/// * `ctx` - Cached searcher context with index and searcher +/// * `query_ast_json` - Quickwit QueryAst JSON string +/// +/// # Returns +/// Vec of (segment_ord, doc_id) pairs for all matching documents. +pub async fn perform_bulk_search( + ctx: &CachedSearcherContext, + query_ast_json: &str, +) -> Result> { + let t0 = std::time::Instant::now(); + + // ======================================================================== + // Query rewriting for companion splits (must happen before query building) + // ======================================================================== + // The regular search() path rewrites queries for: + // 1. FieldPresence on hash-optimized fields → _phash_* fields + // 2. Term queries on exact_only fields → _phash_ hash lookups + // Without these rewrites, IS NOT NULL and exact_only EqualTo queries fail. + let effective_json = rewrite_companion_query(ctx, query_ast_json)?; + let effective_json_str = effective_json.as_deref().unwrap_or(query_ast_json); + + // Parse QueryAst from (possibly rewritten) JSON + let query_ast: QueryAst = serde_json::from_str(effective_json_str) + .with_context(|| format!("Failed to parse QueryAst JSON: {}", &effective_json_str[..effective_json_str.len().min(200)]))?; + + // Get schema and tokenizer manager for query building + let schema = ctx.cached_index.schema(); + let tokenizer_manager = create_default_quickwit_tokenizer_manager(); + + // Convert QueryAst → tantivy Query + // with_validation=false to silently handle fields not in schema + let tantivy_query = query_ast + .build_tantivy_query(&schema, &tokenizer_manager, &[], false) + .map_err(|e| anyhow!("Failed to build tantivy query: {}", e))?; + + perf_println!( + "⏱️ BULK_SEARCH: query parse + build took {}ms", + t0.elapsed().as_millis() + ); + + // ======================================================================== + // Warm up: term dictionaries, posting lists, fast fields + // ======================================================================== + // The HotDirectory only caches a subset of the split data; term, postings, and + // fast files may not be in the hot cache. Tantivy's searcher.search() is synchronous + // and will fail with "StorageDirectory only supports async reads" if it hits + // cold data. The warm_up calls load the data into the HotDirectory cache + // asynchronously, after which the synchronous search succeeds. + let t_warmup = std::time::Instant::now(); + + // Extract fields that need fast field access (range, field_presence/exists queries) + // from the REWRITTEN query (so _phash_* fields are included) + let fast_field_names = crate::parquet_companion::field_extraction::extract_range_query_fields(effective_json_str); + + let mut warmup_futures = Vec::new(); + for segment_reader in ctx.cached_searcher.segment_readers() { + for (field, field_entry) in schema.fields() { + if !field_entry.is_indexed() { + continue; + } + if let Ok(inverted_index) = segment_reader.inverted_index(field) { + let inv = inverted_index.clone(); + warmup_futures.push(async move { + // Warm term dictionary (FST) + let _ = inv.terms().warm_up_dictionary().await; + // Warm posting lists + let _ = inv.warm_postings_full(false).await; + }); + } + } + } + if !warmup_futures.is_empty() { + futures::future::join_all(warmup_futures).await; + perf_println!( + "⏱️ BULK_SEARCH: warmup term+postings took {}ms", + t_warmup.elapsed().as_millis() + ); + } + + // Warm fast fields for fields that need them (range, exists, _phash_* queries) + if !fast_field_names.is_empty() { + let field_names: Vec = fast_field_names.into_iter().collect(); + perf_println!("⏱️ BULK_SEARCH: warming fast fields for {:?}", field_names); + let _ = ctx.warm_native_fast_fields_l1_for_fields(&field_names).await; + perf_println!( + "⏱️ BULK_SEARCH: warmup fast fields took {}ms", + t_warmup.elapsed().as_millis() + ); + } + + // For companion splits with an augmented directory, ensure parquet-sourced fast + // fields are transcoded and available. This mirrors what ensure_fast_fields_for_query() + // does in the regular search() path. + if ctx.augmented_directory.is_some() { + let _ = crate::split_searcher::async_impl::ensure_fast_fields_for_query( + ctx, effective_json_str, None, + ).await?; + perf_println!( + "⏱️ BULK_SEARCH: ensure_fast_fields took {}ms", + t_warmup.elapsed().as_millis() + ); + } + + // Execute search with DocIdCollector (no scoring) + let t_search = std::time::Instant::now(); + let doc_ids = ctx + .cached_searcher + .search(&*tantivy_query, &DocIdCollector) + .map_err(|e| anyhow!("DocIdCollector search failed: {}", e))?; + + perf_println!( + "⏱️ BULK_SEARCH: search returned {} docs in {}ms (no scoring)", + doc_ids.len(), + t_search.elapsed().as_millis() + ); + + Ok(doc_ids) +} + +/// Apply companion-mode query rewrites (hash field presence + exact_only string indexing). +/// +/// Returns `Ok(Some(rewritten))` if any rewrite was applied, `Ok(None)` if no changes needed. +fn rewrite_companion_query( + ctx: &CachedSearcherContext, + query_json: &str, +) -> Result> { + let manifest = match ctx.parquet_manifest.as_ref() { + Some(m) => m, + None => return Ok(None), // Not a companion split — no rewrites needed + }; + + let mut current = query_json.to_string(); + let mut changed = false; + + // Rewrite FieldPresence (exists/IS NOT NULL) queries on string hash fields + // to target _phash_* U64 fields instead of the original string field + if !manifest.string_hash_fields.is_empty() { + if let Some(rewritten) = crate::parquet_companion::hash_field_rewriter::rewrite_query_for_hash_fields( + ¤t, + &manifest.string_hash_fields, + ) { + perf_println!("⏱️ BULK_SEARCH: rewrote FieldPresence → hash field(s)"); + current = rewritten; + changed = true; + } + } + + // Rewrite term queries for exact_only / text_*_exactonly compact string indexing + // (converts field term queries to _phash_ hash lookups) + if !manifest.string_indexing_modes.is_empty() { + match crate::parquet_companion::hash_field_rewriter::rewrite_query_for_string_indexing( + ¤t, + &manifest.string_indexing_modes, + )? { + Some(rewritten) => { + perf_println!("⏱️ BULK_SEARCH: rewrote query for compact string indexing mode(s)"); + current = rewritten; + changed = true; + } + None => {} + } + } + + Ok(if changed { Some(current) } else { None }) +} + +/// Resolve doc addresses to parquet file groups entirely in Rust. +/// +/// Uses the __pq_file_hash and __pq_row_in_file fast field columns for O(1) +/// per-doc resolution. Groups results by parquet file index, sorted by +/// row_in_file within each group for sequential I/O. +/// +/// This is equivalent to `resolve_doc_addresses_to_groups` in doc_retrieval_jni.rs +/// but extracted as a standalone function for reuse by both fused and streaming paths. +/// +/// # Arguments +/// * `ctx` - Cached searcher context with __pq column handles +/// * `doc_ids` - Slice of (segment_ord, doc_id) pairs from DocIdCollector +/// +/// # Returns +/// HashMap from file_idx → Vec<(original_index, row_in_file)>, sorted by row_in_file. +/// +/// # Fallback +/// When `has_merge_safe_tracking` is false (legacy splits without __pq fields), +/// falls back to manifest-based positional resolution via `group_doc_addresses_by_file`. +pub async fn resolve_to_parquet_locations( + ctx: &Arc, + doc_ids: &[(u32, u32)], +) -> Result>> { + if doc_ids.is_empty() { + return Ok(HashMap::new()); + } + + // Fallback for legacy splits without __pq fast fields + if !ctx.has_merge_safe_tracking { + perf_println!("⏱️ BULK_RESOLVE: using legacy manifest-based resolution"); + let manifest = ctx + .parquet_manifest + .as_ref() + .ok_or_else(|| anyhow!("no parquet manifest for legacy resolution"))?; + return crate::parquet_companion::docid_mapping::group_doc_addresses_by_file( + doc_ids, manifest, + ) + .map_err(|e| anyhow!("{}", e)); + } + + let t0 = std::time::Instant::now(); + + // Ensure __pq columns loaded for all referenced segments + let unique_segments: HashSet = doc_ids.iter().map(|&(seg, _)| seg).collect(); + for &seg_ord in &unique_segments { + ctx.ensure_pq_segment_loaded(seg_ord) + .await + .with_context(|| format!("Failed to load __pq fields for segment {}", seg_ord))?; + } + + perf_println!( + "⏱️ BULK_RESOLVE: loaded __pq columns for {} segments in {}ms", + unique_segments.len(), + t0.elapsed().as_millis() + ); + + // Resolve each doc to (file_idx, row_in_file) via fast fields + let t_resolve = std::time::Instant::now(); + let pq_cols = ctx.pq_columns.read() + .map_err(|e| anyhow!("Failed to acquire __pq columns read lock: {}", e))?; + let mut resolved: Vec<(usize, u64, u64)> = Vec::with_capacity(doc_ids.len()); + + for (original_idx, &(seg_ord, doc_id)) in doc_ids.iter().enumerate() { + let (fh_col, row_col) = pq_cols.get(seg_ord as usize) + .and_then(|opt| opt.as_ref()) + .ok_or_else(|| anyhow!("__pq columns not loaded for segment {}", seg_ord))?; + + let file_hash = fh_col + .values_for_doc(doc_id) + .next() + .ok_or_else(|| anyhow!("no __pq_file_hash for seg={} doc={}", seg_ord, doc_id))?; + let row_in_file = row_col + .values_for_doc(doc_id) + .next() + .ok_or_else(|| anyhow!("no __pq_row_in_file for seg={} doc={}", seg_ord, doc_id))?; + + resolved.push((original_idx, file_hash, row_in_file)); + } + drop(pq_cols); // Release read lock + + perf_println!( + "⏱️ BULK_RESOLVE: resolved {} docs via fast fields in {}ms", + resolved.len(), + t_resolve.elapsed().as_millis() + ); + + // Group by file using the pre-built hash→index lookup + let t_group = std::time::Instant::now(); + let groups = crate::parquet_companion::docid_mapping::group_resolved_locations_by_file( + &resolved, + &ctx.parquet_file_hash_index, + ) + .map_err(|e| anyhow!("{}", e))?; + + perf_println!( + "⏱️ BULK_RESOLVE: grouped into {} files in {}ms", + groups.len(), + t_group.elapsed().as_millis() + ); + + Ok(groups) +} + +/// Get the parquet storage from the context, with helpful error messages. +pub fn get_parquet_storage( + ctx: &CachedSearcherContext, +) -> Result> { + match ctx.parquet_storage.as_ref() { + Some(s) => Ok(s.clone()), + None => { + let reason = if ctx.parquet_table_root.is_none() { + "parquet_table_root was not set. Pass the table root path to createSplitSearcher() \ + or configure it via CacheConfig.withParquetTableRoot()." + } else { + "parquet storage creation failed (likely bad credentials or unreachable endpoint). \ + Enable TANTIVY4JAVA_DEBUG=1 and check stderr for the storage creation error." + }; + Err(anyhow!( + "Parquet companion retrieval failed: {}", + reason + )) + } + } +} + diff --git a/native/src/split_searcher/docid_collector.rs b/native/src/split_searcher/docid_collector.rs new file mode 100644 index 0000000..fb2b747 --- /dev/null +++ b/native/src/split_searcher/docid_collector.rs @@ -0,0 +1,153 @@ +// docid_collector.rs - No-score bulk document ID collector +// +// Custom tantivy Collector that collects (segment_ord, doc_id) pairs without +// computing BM25 scores. Uses tantivy's no-score batch path (collect_block +// with 64-doc chunks) for maximum throughput. +// +// Memory: 8 bytes per hit (2x u32). 2M hits = 16MB. +// +// This collector is the foundation for both: +// - Fused search+retrieve (Phase 0): small-to-medium companion queries +// - Streaming retrieval (Phase 3): large result set companion queries + +use tantivy::collector::{Collector, SegmentCollector}; +use tantivy::{DocId, Score, SegmentOrdinal, SegmentReader}; + +/// Collector that returns all matching doc addresses without scoring. +/// +/// Returns `requires_scoring() = false`, which tells tantivy to: +/// 1. Skip BM25 term-frequency decompression (biggest CPU savings) +/// 2. Use `for_each_no_score()` → `collect_block()` with 64-doc batches +/// 3. Pass score=0.0 (ignored) +/// +/// Use this instead of Quickwit's leaf_search path when you need all matching +/// doc IDs and don't care about relevance scores (companion mode row retrieval). +pub struct DocIdCollector; + +impl Collector for DocIdCollector { + type Fruit = Vec<(u32, u32)>; // (segment_ord, doc_id) + type Child = DocIdSegmentCollector; + + fn for_segment( + &self, + segment_local_id: SegmentOrdinal, + _reader: &SegmentReader, + ) -> tantivy::Result { + Ok(DocIdSegmentCollector { + segment_ord: segment_local_id, + doc_ids: Vec::new(), + }) + } + + fn requires_scoring(&self) -> bool { + false + } + + fn merge_fruits( + &self, + segment_fruits: Vec>, + ) -> tantivy::Result { + let total: usize = segment_fruits.iter() + .map(|v| v.len()) + .try_fold(0usize, |acc, len| acc.checked_add(len)) + .ok_or_else(|| tantivy::TantivyError::InternalError( + "Result set too large: document count overflow".to_string() + ))?; + let mut result = Vec::with_capacity(total); + for fruit in segment_fruits { + result.extend(fruit); + } + Ok(result) + } +} + +pub struct DocIdSegmentCollector { + segment_ord: u32, + doc_ids: Vec<(u32, u32)>, +} + +impl SegmentCollector for DocIdSegmentCollector { + type Fruit = Vec<(u32, u32)>; + + fn collect(&mut self, doc: DocId, _score: Score) { + self.doc_ids.push((self.segment_ord, doc)); + } + + /// Batch collection — called with up to 64 doc IDs at a time when + /// requires_scoring() is false and no alive_bitset filtering is needed. + fn collect_block(&mut self, docs: &[DocId]) { + self.doc_ids.reserve(docs.len()); + for &doc in docs { + self.doc_ids.push((self.segment_ord, doc)); + } + } + + fn harvest(self) -> Vec<(u32, u32)> { + self.doc_ids + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_docid_collector_requires_no_scoring() { + let collector = DocIdCollector; + assert!(!collector.requires_scoring()); + } + + #[test] + fn test_segment_collector_collect() { + let mut seg_collector = DocIdSegmentCollector { + segment_ord: 2, + doc_ids: Vec::new(), + }; + seg_collector.collect(10, 0.0); + seg_collector.collect(20, 1.5); // score ignored + seg_collector.collect(30, 0.0); + + let result = seg_collector.harvest(); + assert_eq!(result, vec![(2, 10), (2, 20), (2, 30)]); + } + + #[test] + fn test_segment_collector_collect_block() { + let mut seg_collector = DocIdSegmentCollector { + segment_ord: 0, + doc_ids: Vec::new(), + }; + seg_collector.collect_block(&[100, 101, 102, 103]); + seg_collector.collect_block(&[200, 201]); + + let result = seg_collector.harvest(); + assert_eq!( + result, + vec![(0, 100), (0, 101), (0, 102), (0, 103), (0, 200), (0, 201)] + ); + } + + #[test] + fn test_merge_fruits_combines_segments() { + let collector = DocIdCollector; + let fruits = vec![ + vec![(0, 10), (0, 20)], + vec![(1, 5), (1, 15), (1, 25)], + vec![(2, 100)], + ]; + let merged = collector.merge_fruits(fruits).unwrap(); + assert_eq!(merged.len(), 6); + assert_eq!(merged[0], (0, 10)); + assert_eq!(merged[2], (1, 5)); + assert_eq!(merged[5], (2, 100)); + } + + #[test] + fn test_merge_fruits_empty() { + let collector = DocIdCollector; + let fruits: Vec> = vec![vec![], vec![], vec![]]; + let merged = collector.merge_fruits(fruits).unwrap(); + assert!(merged.is_empty()); + } + +} diff --git a/native/src/split_searcher/document_retrieval/doc_retrieval_jni.rs b/native/src/split_searcher/document_retrieval/doc_retrieval_jni.rs index 6183e44..0f34752 100644 --- a/native/src/split_searcher/document_retrieval/doc_retrieval_jni.rs +++ b/native/src/split_searcher/document_retrieval/doc_retrieval_jni.rs @@ -1166,3 +1166,257 @@ pub extern "system" fn Java_io_indextables_tantivy4java_split_SplitSearcher_nati } }).unwrap_or(-1) } + +// ===================================================================== +// Streaming Retrieval Session JNI Methods +// ===================================================================== + +/// Start a streaming bulk retrieval session for companion mode. +/// +/// Performs the search + resolve phases synchronously, then starts a background +/// producer task that streams batches through a bounded channel. +/// +/// Returns a session handle (jlong) that Java uses to poll batches via +/// nativeNextBatch. Returns 0 on error (exception thrown) or if not a companion split. +#[no_mangle] +pub extern "system" fn Java_io_indextables_tantivy4java_split_SplitSearcher_nativeStartStreamingRetrieval( + mut env: JNIEnv, + _class: JClass, + searcher_ptr: jlong, + query_ast_json: jni::objects::JString, + field_names: jni::sys::jobjectArray, +) -> jlong { + if searcher_ptr == 0 { + to_java_exception(&mut env, &anyhow::anyhow!("Invalid searcher pointer")); + return 0; + } + + // Check if this split has a parquet manifest (companion vs regular split) + let has_manifest = with_arc_safe(searcher_ptr, |ctx: &Arc| { + ctx.parquet_manifest.is_some() + }) + .unwrap_or(false); + + use crate::utils::convert_throwable; + convert_throwable(&mut env, |env| { + // Extract query string + let query_json: String = env + .get_string(&query_ast_json) + .map_err(|e| anyhow::anyhow!("Failed to get query string: {}", e))? + .into(); + + // Extract field names + let projected_fields = extract_jni_field_names(env, field_names); + + let t_total = std::time::Instant::now(); + perf_println!( + "⏱️ STREAMING_JNI: === nativeStartStreamingRetrieval START === companion={} fields={:?}", + has_manifest, projected_fields + ); + + let result = with_arc_safe(searcher_ptr, |ctx: &Arc| { + let runtime = crate::runtime_manager::QuickwitRuntimeManager::global().handle(); + let _guard = runtime.enter(); + + tokio::task::block_in_place(|| { + runtime.block_on(async { + // Phase 1: No-score search (works for both companion and regular splits) + let doc_ids = crate::split_searcher::bulk_retrieval::perform_bulk_search( + ctx, &query_json, + ).await?; + + perf_println!("⏱️ STREAMING_JNI: search found {} docs", doc_ids.len()); + + if has_manifest { + // === COMPANION PATH: parquet-based streaming === + if doc_ids.is_empty() { + perf_println!("⏱️ STREAMING_JNI: no matches — returning empty session (companion)"); + let (_, rx) = tokio::sync::mpsc::channel::>(1); + let manifest = ctx.parquet_manifest.as_ref().unwrap(); + let schema = crate::parquet_companion::streaming_ffi::build_tantivy_schema_pub( + manifest, projected_fields.as_deref(), + )?; + let session = crate::parquet_companion::streaming_ffi::StreamingRetrievalSession::new_empty( + rx, schema, + ); + let handle = crate::utils::arc_to_jlong(std::sync::Arc::new(std::sync::Mutex::new(session))); + return Ok::(handle); + } + + // Phase 2: Resolve to parquet locations + let groups = crate::split_searcher::bulk_retrieval::resolve_to_parquet_locations( + ctx, &doc_ids, + ).await?; + + perf_println!( + "⏱️ STREAMING_JNI: resolved to {} file groups (companion)", + groups.len() + ); + + // Phase 3: Start companion streaming producer + let manifest = ctx.parquet_manifest.as_ref().unwrap().clone(); + let storage = crate::split_searcher::bulk_retrieval::get_parquet_storage(ctx)?; + + let session = crate::parquet_companion::streaming_ffi::start_streaming_retrieval( + groups, + projected_fields, + manifest, + storage, + Some(ctx.parquet_metadata_cache.clone()), + Some(ctx.parquet_byte_range_cache.clone()), + ctx.parquet_coalesce_config, + )?; + + let handle = crate::utils::arc_to_jlong(std::sync::Arc::new(std::sync::Mutex::new(session))); + Ok(handle) + } else { + // === REGULAR PATH: tantivy doc store streaming === + let session = crate::split_searcher::streaming_doc_retrieval::start_tantivy_streaming_retrieval( + ctx, doc_ids, projected_fields, + )?; + + let handle = crate::utils::arc_to_jlong(std::sync::Arc::new(std::sync::Mutex::new(session))); + Ok(handle) + } + }) + }) + }); + + match result { + Some(Ok(handle)) => { + perf_println!( + "⏱️ STREAMING_JNI: === session started, took {}ms ===", + t_total.elapsed().as_millis() + ); + Ok(handle) + } + Some(Err(e)) => { + perf_println!( + "⏱️ STREAMING_JNI: === FAILED after {}ms: {} ===", + t_total.elapsed().as_millis(), e + ); + Err(anyhow::anyhow!("Failed to start streaming retrieval: {}", e)) + } + None => Err(anyhow::anyhow!("Searcher context not found")), + } + }) + .unwrap_or(0) +} + +/// Poll the next batch from a streaming session. +/// +/// Writes Arrow FFI data to the provided addresses. +/// Returns: >0 = row count, 0 = end of stream, -1 = error. +#[no_mangle] +pub extern "system" fn Java_io_indextables_tantivy4java_split_SplitSearcher_nativeNextBatch( + mut env: JNIEnv, + _class: JClass, + session_ptr: jlong, + array_addrs: jni::sys::jlongArray, + schema_addrs: jni::sys::jlongArray, +) -> jint { + if session_ptr == 0 { + to_java_exception(&mut env, &anyhow::anyhow!("Invalid session pointer")); + return -1; + } + + use crate::utils::convert_throwable; + convert_throwable(&mut env, |env| { + // Look up session in registry (safe — no raw pointer dereference) + let session_arc = crate::utils::jlong_to_arc::>(session_ptr) + .ok_or_else(|| anyhow::anyhow!("Streaming session not found or already closed (handle={})", session_ptr))?; + + // Extract FFI addresses + let array_addrs_jni = unsafe { jni::objects::JLongArray::from_raw(array_addrs) }; + let schema_addrs_jni = unsafe { jni::objects::JLongArray::from_raw(schema_addrs) }; + + let addrs_len = env + .get_array_length(&array_addrs_jni) + .map_err(|e| anyhow::anyhow!("Failed to get array_addrs length: {}", e))? + as usize; + + let mut array_addrs_vec = vec![0i64; addrs_len]; + env.get_long_array_region(&array_addrs_jni, 0, &mut array_addrs_vec) + .map_err(|e| anyhow::anyhow!("Failed to get array_addrs: {}", e))?; + + let mut schema_addrs_vec = vec![0i64; addrs_len]; + env.get_long_array_region(&schema_addrs_jni, 0, &mut schema_addrs_vec) + .map_err(|e| anyhow::anyhow!("Failed to get schema_addrs: {}", e))?; + + // Poll next batch from channel (blocking) + let mut session = session_arc.lock() + .map_err(|e| anyhow::anyhow!("Failed to lock streaming session: {}", e))?; + match session.blocking_next() { + Some(Ok(batch)) => { + let row_count = crate::parquet_companion::streaming_ffi::write_batch_to_ffi( + &batch, + &array_addrs_vec, + &schema_addrs_vec, + )?; + perf_println!("⏱️ STREAMING_JNI: nextBatch returned {} rows", row_count); + Ok(row_count as jint) + } + Some(Err(e)) => { + perf_println!("⏱️ STREAMING_JNI: nextBatch error: {}", e); + Err(anyhow::anyhow!("Streaming batch read failed: {}", e)) + } + None => { + perf_println!("⏱️ STREAMING_JNI: end of stream"); + Ok(0) // End of stream + } + } + }) + .unwrap_or(-1) +} + +/// Close and free a streaming session. +/// +/// Safe to call multiple times — second call is a no-op (registry entry already removed). +/// Must be called to release native resources and stop the producer task. +#[no_mangle] +pub extern "system" fn Java_io_indextables_tantivy4java_split_SplitSearcher_nativeCloseStreamingSession( + _env: JNIEnv, + _class: JClass, + session_ptr: jlong, +) { + if session_ptr != 0 { + perf_println!("⏱️ STREAMING_JNI: closing session (handle={})", session_ptr); + crate::utils::release_arc(session_ptr); + } +} + +/// Get the number of columns in a streaming session's output schema. +/// Used by Java to allocate the correct number of FFI address arrays. +#[no_mangle] +pub extern "system" fn Java_io_indextables_tantivy4java_split_SplitSearcher_nativeGetStreamingColumnCount( + mut env: JNIEnv, + _class: JClass, + session_ptr: jlong, +) -> jint { + if session_ptr == 0 { + to_java_exception(&mut env, &anyhow::anyhow!("Invalid session pointer")); + return -1; + } + + let session_arc = crate::utils::jlong_to_arc::>(session_ptr); + + match session_arc { + Some(arc) => { + match arc.lock() { + Ok(session) => session.num_columns() as jint, + Err(_) => { + to_java_exception(&mut env, &anyhow::anyhow!("Failed to lock streaming session")); + -1 + } + } + } + None => { + to_java_exception(&mut env, &anyhow::anyhow!("Streaming session not found or already closed")); + -1 + } + } +} diff --git a/native/src/split_searcher/mod.rs b/native/src/split_searcher/mod.rs index 125eeb7..1ccda31 100644 --- a/native/src/split_searcher/mod.rs +++ b/native/src/split_searcher/mod.rs @@ -17,6 +17,10 @@ pub mod jni_utils; pub mod jni_agg_arrow; pub mod async_impl; pub mod aggregation_arrow_ffi; +pub mod docid_collector; +pub mod bulk_retrieval; +pub mod streaming_doc_retrieval; + // Re-exports - items used by other modules in the crate pub(crate) use types::CachedSearcherContext; diff --git a/native/src/split_searcher/streaming_doc_retrieval.rs b/native/src/split_searcher/streaming_doc_retrieval.rs new file mode 100644 index 0000000..3b726d1 --- /dev/null +++ b/native/src/split_searcher/streaming_doc_retrieval.rs @@ -0,0 +1,538 @@ +// streaming_doc_retrieval.rs - Streaming Arrow FFI retrieval for non-companion (regular) splits +// +// Converts Tantivy doc store documents to Arrow RecordBatches and streams them +// through the same StreamingRetrievalSession / mpsc::channel(2) architecture +// used by the companion (parquet) path. +// +// Architecture: +// perform_bulk_search() → Vec<(segment_ord, doc_id)> +// start_tantivy_streaming_retrieval() → spawns tokio producer task +// producer retrieves docs in BATCH_SIZE chunks via doc_async() +// converts each chunk to RecordBatch → sends through channel +// consumer calls next_batch() (same as companion path) +// +// Memory bound: same ~24MB peak as companion path (2 × batch in channel). + +use std::sync::Arc; + +use anyhow::{Context, Result}; +use arrow_array::builder::*; +use arrow_array::RecordBatch; +use arrow_schema::{DataType, Field, Schema, TimeUnit}; +use tantivy::schema::{FieldEntry, FieldType, OwnedValue}; +use tokio::sync::mpsc; + +use crate::parquet_companion::streaming_ffi::StreamingRetrievalSession; +use crate::perf_println; +use super::types::CachedSearcherContext; + +/// Batch size for doc store retrieval. Smaller than companion's 128K because +/// doc store is row-oriented (each doc_async is an I/O op). +const TANTIVY_BATCH_SIZE: usize = 4096; + +/// Max concurrent doc_async calls per batch. +const DOC_ASYNC_CONCURRENCY: usize = 64; + +/// Build an Arrow schema from a Tantivy schema, optionally filtered by projected fields. +/// +/// Only includes stored fields (non-stored fields are not in the doc store). +/// Returns the schema and a vec of (tantivy Field, arrow column index) for efficient lookup. +pub fn tantivy_schema_to_arrow( + schema: &tantivy::schema::Schema, + projected_fields: Option<&[String]>, +) -> Result<(Arc, Vec<(tantivy::schema::Field, String)>)> { + let mut arrow_fields = Vec::new(); + let mut field_mapping = Vec::new(); + + for (field, field_entry) in schema.fields() { + let field_name = field_entry.name().to_string(); + + // Filter by projected fields if specified + if let Some(proj) = projected_fields { + if !proj.iter().any(|p| p == &field_name) { + continue; + } + } + + // Only include stored fields + if !field_entry.is_stored() { + continue; + } + + let arrow_type = tantivy_field_type_to_arrow(field_entry); + arrow_fields.push(Field::new(&field_name, arrow_type, true)); + field_mapping.push((field, field_name)); + } + + if arrow_fields.is_empty() { + anyhow::bail!("No stored fields found for streaming schema (check field projection)"); + } + + Ok((Arc::new(Schema::new(arrow_fields)), field_mapping)) +} + +/// Map a Tantivy FieldEntry to an Arrow DataType. +fn tantivy_field_type_to_arrow(entry: &FieldEntry) -> DataType { + match entry.field_type() { + FieldType::Str(_) => DataType::Utf8, + FieldType::I64(_) => DataType::Int64, + FieldType::U64(_) => DataType::UInt64, + FieldType::F64(_) => DataType::Float64, + FieldType::Bool(_) => DataType::Boolean, + FieldType::Date(_) => DataType::Timestamp(TimeUnit::Microsecond, None), + FieldType::Bytes(_) => DataType::Binary, + FieldType::IpAddr(_) => DataType::Utf8, + FieldType::JsonObject(_) => DataType::Utf8, + FieldType::Facet(_) => DataType::Utf8, + } +} + +/// Start a streaming retrieval session for non-companion (regular Tantivy) splits. +/// +/// Uses the same StreamingRetrievalSession infrastructure as companion mode. +/// The producer retrieves documents from Tantivy's doc store in batches, +/// converts them to Arrow RecordBatches, and streams through an mpsc channel. +pub fn start_tantivy_streaming_retrieval( + ctx: &Arc, + doc_ids: Vec<(u32, u32)>, + projected_fields: Option>, +) -> Result { + let schema = ctx.cached_searcher.schema(); + let (arrow_schema, field_mapping) = + tantivy_schema_to_arrow(&schema, projected_fields.as_deref())?; + + if doc_ids.is_empty() { + // Empty session — same pattern as companion path + let (_, rx) = mpsc::channel::>(1); + return Ok(StreamingRetrievalSession::new_empty(rx, arrow_schema)); + } + + let num_docs = doc_ids.len(); + let (tx, rx) = mpsc::channel::>(2); + + let searcher = ctx.cached_searcher.clone(); + let arrow_schema_clone = arrow_schema.clone(); + + let handle = tokio::spawn(async move { + let tx_err = tx.clone(); + let result = produce_tantivy_batches( + tx, + doc_ids, + &searcher, + &arrow_schema_clone, + &field_mapping, + ) + .await; + + if let Err(e) = result { + let _ = tx_err.send(Err(e)).await; + } + }); + + perf_println!( + "⏱️ TANTIVY_STREAMING: session started — {} docs, {} columns", + num_docs, + arrow_schema.fields().len() + ); + + Ok(StreamingRetrievalSession::new(rx, arrow_schema, handle)) +} + +/// Producer: retrieves Tantivy documents in batches, converts to RecordBatch, +/// sends through channel. +async fn produce_tantivy_batches( + tx: mpsc::Sender>, + mut doc_ids: Vec<(u32, u32)>, + searcher: &Arc, + arrow_schema: &Arc, + field_mapping: &[(tantivy::schema::Field, String)], +) -> Result<()> { + let t_total = std::time::Instant::now(); + let total_docs = doc_ids.len(); + + perf_println!( + "⏱️ TANTIVY_STREAMING: producer start — {} total docs", + total_docs + ); + + // Sort by (segment_ord, doc_id) for cache locality + doc_ids.sort(); + + let num_segments = searcher.segment_readers().len(); + let mut rows_emitted = 0usize; + let mut batches_sent = 0usize; + + // Process in chunks of TANTIVY_BATCH_SIZE + for chunk in doc_ids.chunks(TANTIVY_BATCH_SIZE) { + let t_batch = std::time::Instant::now(); + + // Retrieve documents concurrently using buffered stream + let docs = retrieve_docs_concurrent(chunk, searcher, num_segments).await?; + + // Convert to RecordBatch + let batch = docs_to_record_batch(&docs, arrow_schema, field_mapping)?; + let batch_rows = batch.num_rows(); + + // Send through channel (blocks if consumer is behind) + if tx.send(Ok(batch)).await.is_err() { + perf_println!( + "⏱️ TANTIVY_STREAMING: consumer dropped after {} rows — stopping", + rows_emitted + ); + return Ok(()); + } + + rows_emitted += batch_rows; + batches_sent += 1; + + perf_println!( + "⏱️ TANTIVY_STREAMING: batch {} — {} rows in {}ms", + batches_sent, + batch_rows, + t_batch.elapsed().as_millis() + ); + } + + perf_println!( + "⏱️ TANTIVY_STREAMING: producer complete — {} batches, {} rows, took {}ms", + batches_sent, + rows_emitted, + t_total.elapsed().as_millis() + ); + + Ok(()) +} + +/// Retrieve a chunk of documents concurrently using buffered futures. +async fn retrieve_docs_concurrent( + chunk: &[(u32, u32)], + searcher: &Arc, + num_segments: usize, +) -> Result> { + use futures::stream::{StreamExt, TryStreamExt}; + + // Collect into owned vec to avoid lifetime issues with async closures + let owned_ids: Vec<(u32, u32)> = chunk.to_vec(); + + let doc_futures = owned_ids.into_iter().map(move |(seg_ord, doc_id)| { + let s = searcher.clone(); + async move { + if seg_ord as usize >= num_segments { + return Err(anyhow::anyhow!( + "Invalid segment ordinal {}: index has {} segment(s)", + seg_ord, + num_segments + )); + } + let addr = tantivy::DocAddress::new(seg_ord, doc_id); + tokio::time::timeout(std::time::Duration::from_secs(10), s.doc_async(addr)) + .await + .map_err(|_| anyhow::anyhow!("doc_async timed out for {:?}", addr))? + .map_err(|e| anyhow::anyhow!("doc_async failed for {:?}: {}", addr, e)) + } + }); + + futures::stream::iter(doc_futures) + .buffered(DOC_ASYNC_CONCURRENCY) + .try_collect::>() + .await +} + +/// Convert a batch of TantivyDocuments to an Arrow RecordBatch. +/// +/// For each column in the arrow schema, iterates all docs and extracts the first +/// matching field value. Missing values become nulls. +fn docs_to_record_batch( + docs: &[tantivy::TantivyDocument], + arrow_schema: &Arc, + field_mapping: &[(tantivy::schema::Field, String)], +) -> Result { + let num_rows = docs.len(); + let num_cols = field_mapping.len(); + let mut columns: Vec = Vec::with_capacity(num_cols); + + for (col_idx, (tantivy_field, _field_name)) in field_mapping.iter().enumerate() { + let arrow_type = arrow_schema.field(col_idx).data_type(); + let col = build_arrow_column(docs, *tantivy_field, arrow_type, num_rows)?; + columns.push(col); + } + + RecordBatch::try_new(arrow_schema.clone(), columns) + .context("Failed to create RecordBatch from Tantivy documents") +} + +/// Build a single Arrow column from the specified field across all documents. +fn build_arrow_column( + docs: &[tantivy::TantivyDocument], + field: tantivy::schema::Field, + arrow_type: &DataType, + num_rows: usize, +) -> Result { + match arrow_type { + DataType::Utf8 => { + let mut builder = StringBuilder::with_capacity(num_rows, num_rows * 32); + for doc in docs { + match find_first_value(doc, field) { + Some(OwnedValue::Str(s)) => builder.append_value(&s), + Some(OwnedValue::IpAddr(ip)) => { + builder.append_value(canonical_ip_string(&ip)) + } + Some(OwnedValue::Facet(f)) => builder.append_value(f.to_path_string()), + Some(OwnedValue::PreTokStr(pts)) => builder.append_value(&pts.text), + Some(OwnedValue::Object(obj)) => { + let json = owned_value_to_json_string(&OwnedValue::Object(obj)); + builder.append_value(&json); + } + Some(OwnedValue::Array(arr)) => { + let json = owned_value_to_json_string(&OwnedValue::Array(arr)); + builder.append_value(&json); + } + Some(OwnedValue::Null) | None => builder.append_null(), + Some(other) => { + // Fallback: convert to string + builder.append_value(format!("{:?}", other)); + } + } + } + Ok(Arc::new(builder.finish())) + } + DataType::Int64 => { + let mut builder = Int64Builder::with_capacity(num_rows); + for doc in docs { + match find_first_value(doc, field) { + Some(OwnedValue::I64(v)) => builder.append_value(v), + Some(OwnedValue::U64(v)) => builder.append_value(v as i64), + _ => builder.append_null(), + } + } + Ok(Arc::new(builder.finish())) + } + DataType::UInt64 => { + let mut builder = UInt64Builder::with_capacity(num_rows); + for doc in docs { + match find_first_value(doc, field) { + Some(OwnedValue::U64(v)) => builder.append_value(v), + Some(OwnedValue::I64(v)) => builder.append_value(v as u64), + _ => builder.append_null(), + } + } + Ok(Arc::new(builder.finish())) + } + DataType::Float64 => { + let mut builder = Float64Builder::with_capacity(num_rows); + for doc in docs { + match find_first_value(doc, field) { + Some(OwnedValue::F64(v)) => builder.append_value(v), + _ => builder.append_null(), + } + } + Ok(Arc::new(builder.finish())) + } + DataType::Boolean => { + let mut builder = BooleanBuilder::with_capacity(num_rows); + for doc in docs { + match find_first_value(doc, field) { + Some(OwnedValue::Bool(v)) => builder.append_value(v), + _ => builder.append_null(), + } + } + Ok(Arc::new(builder.finish())) + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + let mut builder = TimestampMicrosecondBuilder::with_capacity(num_rows); + for doc in docs { + match find_first_value(doc, field) { + Some(OwnedValue::Date(dt)) => { + let micros = dt.into_timestamp_nanos() / 1_000; + builder.append_value(micros); + } + _ => builder.append_null(), + } + } + Ok(Arc::new(builder.finish())) + } + DataType::Binary => { + let mut builder = BinaryBuilder::with_capacity(num_rows, num_rows * 64); + for doc in docs { + match find_first_value(doc, field) { + Some(OwnedValue::Bytes(b)) => builder.append_value(&b), + _ => builder.append_null(), + } + } + Ok(Arc::new(builder.finish())) + } + _ => { + // Fallback: treat as Utf8 + let mut builder = StringBuilder::with_capacity(num_rows, num_rows * 32); + for doc in docs { + match find_first_value(doc, field) { + Some(val) => builder.append_value(format!("{:?}", val)), + None => builder.append_null(), + } + } + Ok(Arc::new(builder.finish())) + } + } +} + +/// Find the first value for a given field in a TantivyDocument. +fn find_first_value( + doc: &tantivy::TantivyDocument, + field: tantivy::schema::Field, +) -> Option { + for (f, val) in doc.field_values() { + if f == field { + return Some(val.into()); + } + } + None +} + +/// Convert an IPv6 address to its canonical display form. +fn canonical_ip_string(ip: &std::net::Ipv6Addr) -> String { + match ip.to_ipv4_mapped() { + Some(v4) => v4.to_string(), + None => ip.to_string(), + } +} + +/// Convert OwnedValue to a JSON string for Object/Array types. +fn owned_value_to_json_string(value: &OwnedValue) -> String { + let json_value = owned_value_to_json(value); + serde_json::to_string(&json_value).unwrap_or_else(|_| "null".to_string()) +} + +/// Convert OwnedValue to serde_json::Value. +fn owned_value_to_json(value: &OwnedValue) -> serde_json::Value { + match value { + OwnedValue::Str(s) => serde_json::Value::String(s.clone()), + OwnedValue::I64(v) => serde_json::Value::Number((*v).into()), + OwnedValue::U64(v) => serde_json::Value::Number((*v).into()), + OwnedValue::F64(v) => serde_json::Number::from_f64(*v) + .map(serde_json::Value::Number) + .unwrap_or(serde_json::Value::Null), + OwnedValue::Bool(v) => serde_json::Value::Bool(*v), + OwnedValue::Date(dt) => { + let nanos = dt.into_timestamp_nanos(); + serde_json::Value::Number(nanos.into()) + } + OwnedValue::Object(obj) => { + let map: serde_json::Map = obj + .iter() + .map(|(k, v)| (k.clone(), owned_value_to_json(v))) + .collect(); + serde_json::Value::Object(map) + } + OwnedValue::Array(arr) => { + serde_json::Value::Array(arr.iter().map(owned_value_to_json).collect()) + } + OwnedValue::IpAddr(ip) => serde_json::Value::String(canonical_ip_string(ip)), + OwnedValue::Facet(f) => serde_json::Value::String(f.to_path_string()), + OwnedValue::PreTokStr(pts) => serde_json::Value::String(pts.text.clone()), + OwnedValue::Bytes(b) => { + use base64::{engine::general_purpose::STANDARD, Engine}; + serde_json::Value::String(STANDARD.encode(b)) + } + OwnedValue::Null => serde_json::Value::Null, + } +} + +#[cfg(test)] +mod tests { + use super::*; + use tantivy::schema::{SchemaBuilder, STORED, TEXT, FAST}; + + #[test] + fn test_tantivy_schema_to_arrow_basic() { + let mut sb = SchemaBuilder::new(); + sb.add_text_field("title", TEXT | STORED); + sb.add_i64_field("count", STORED | FAST); + sb.add_f64_field("score", STORED); + sb.add_bool_field("active", STORED); + let schema = sb.build(); + + let (arrow_schema, mapping) = tantivy_schema_to_arrow(&schema, None).unwrap(); + assert_eq!(arrow_schema.fields().len(), 4); + assert_eq!(mapping.len(), 4); + + assert_eq!(arrow_schema.field(0).name(), "title"); + assert_eq!(*arrow_schema.field(0).data_type(), DataType::Utf8); + + assert_eq!(arrow_schema.field(1).name(), "count"); + assert_eq!(*arrow_schema.field(1).data_type(), DataType::Int64); + + assert_eq!(arrow_schema.field(2).name(), "score"); + assert_eq!(*arrow_schema.field(2).data_type(), DataType::Float64); + + assert_eq!(arrow_schema.field(3).name(), "active"); + assert_eq!(*arrow_schema.field(3).data_type(), DataType::Boolean); + } + + #[test] + fn test_tantivy_schema_to_arrow_projected() { + let mut sb = SchemaBuilder::new(); + sb.add_text_field("title", TEXT | STORED); + sb.add_i64_field("count", STORED | FAST); + sb.add_f64_field("score", STORED); + let schema = sb.build(); + + let fields = vec!["title".to_string(), "score".to_string()]; + let (arrow_schema, mapping) = + tantivy_schema_to_arrow(&schema, Some(&fields)).unwrap(); + assert_eq!(arrow_schema.fields().len(), 2); + assert_eq!(mapping.len(), 2); + assert_eq!(arrow_schema.field(0).name(), "title"); + assert_eq!(arrow_schema.field(1).name(), "score"); + } + + #[test] + fn test_tantivy_schema_to_arrow_no_stored() { + let mut sb = SchemaBuilder::new(); + // TEXT without STORED — field is indexed but not stored + sb.add_text_field("body", TEXT); + let schema = sb.build(); + + let result = tantivy_schema_to_arrow(&schema, None); + assert!(result.is_err()); // No stored fields → error + } + + #[test] + fn test_docs_to_record_batch() { + let mut sb = SchemaBuilder::new(); + let title_field = sb.add_text_field("title", TEXT | STORED); + let count_field = sb.add_i64_field("count", STORED | FAST); + let schema = sb.build(); + + let (arrow_schema, field_mapping) = tantivy_schema_to_arrow(&schema, None).unwrap(); + + let mut doc1 = tantivy::TantivyDocument::default(); + doc1.add_text(title_field, "hello"); + doc1.add_i64(count_field, 42); + + let mut doc2 = tantivy::TantivyDocument::default(); + doc2.add_text(title_field, "world"); + doc2.add_i64(count_field, 99); + + let docs = vec![doc1, doc2]; + let batch = docs_to_record_batch(&docs, &arrow_schema, &field_mapping).unwrap(); + + assert_eq!(batch.num_rows(), 2); + assert_eq!(batch.num_columns(), 2); + + let title_col = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(title_col.value(0), "hello"); + assert_eq!(title_col.value(1), "world"); + + let count_col = batch + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(count_col.value(0), 42); + assert_eq!(count_col.value(1), 99); + } +} diff --git a/pom.xml b/pom.xml index 4be4398..7410bef 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ io.indextables tantivy4java - 0.32.5 + 0.32.6 jar Tantivy4Java Experimental diff --git a/src/main/java/io/indextables/tantivy4java/split/SplitSearcher.java b/src/main/java/io/indextables/tantivy4java/split/SplitSearcher.java index 1071a9d..403016e 100644 --- a/src/main/java/io/indextables/tantivy4java/split/SplitSearcher.java +++ b/src/main/java/io/indextables/tantivy4java/split/SplitSearcher.java @@ -545,6 +545,217 @@ public int docBatchArrowFfi(DocAddress[] docAddresses, arrayAddrs, schemaAddrs); } + /** + * Fused companion-mode search + retrieval: executes a no-score search and returns + * matching rows as Arrow columnar data. + * + * @deprecated Use {@link #startStreamingRetrieval(String, String...)} instead. + * The streaming path has negligible overhead and is the single + * companion retrieval path going forward. This method now delegates + * to the streaming path internally. + * + * @param queryAstJson Quickwit query AST JSON string + * @param arrayAddrs pre-allocated ArrowArray memory addresses (one per field) + * @param schemaAddrs pre-allocated ArrowSchema memory addresses (one per field) + * @param fields field names to retrieve (null or empty for all fields) + * @return row count (>0 = data, 0 = no matches, -1 = not a companion split) + */ + @Deprecated(forRemoval = true) + public int searchAndRetrieveArrowFfi(String queryAstJson, + long[] arrayAddrs, long[] schemaAddrs, + String... fields) { + if (nativePtr == 0) { + throw new IllegalStateException("SplitSearcher has been closed or not properly initialized"); + } + + // Delegate to streaming path: start session → drain all batches → return total row count + StreamingSession session; + try { + session = startStreamingRetrieval(queryAstJson, fields); + } catch (IllegalStateException e) { + // Not a companion split or query error — return -1 for backward compatibility + return -1; + } + + try { + int totalRows = 0; + while (true) { + int rows = session.nextBatch(arrayAddrs, schemaAddrs); + if (rows <= 0) break; + totalRows += rows; + } + return totalRows; + } finally { + session.close(); + } + } + + // ========================================================================= + // Streaming Retrieval API (companion mode, large result sets > 50K rows) + // ========================================================================= + + /** + * A streaming retrieval session that implements AutoCloseable for safe resource management. + * + *

Wraps a native session handle with double-close protection and lifecycle tracking. + * Use with try-with-resources for automatic cleanup: + * + *

{@code
+     * try (StreamingSession session = searcher.startStreamingRetrieval(queryJson, "field1")) {
+     *     int numCols = session.getColumnCount();
+     *     long[] arrayAddrs = new long[numCols];
+     *     long[] schemaAddrs = new long[numCols];
+     *     int rows;
+     *     while ((rows = session.nextBatch(arrayAddrs, schemaAddrs)) > 0) {
+     *         // process batch...
+     *     }
+     * }
+     * }
+ */ + public static class StreamingSession implements AutoCloseable { + private long handle; + private volatile boolean closed = false; + + StreamingSession(long handle) { + this.handle = handle; + } + + /** + * Poll the next batch from the streaming session. + * + * @param arrayAddrs pre-allocated ArrowArray memory addresses (one per column) + * @param schemaAddrs pre-allocated ArrowSchema memory addresses (one per column) + * @return row count (>0 = data, 0 = end of stream) + * @throws IllegalStateException if session is closed + */ + public synchronized int nextBatch(long[] arrayAddrs, long[] schemaAddrs) { + if (closed) { + throw new IllegalStateException("Streaming session has been closed"); + } + return nativeNextBatch(handle, arrayAddrs, schemaAddrs); + } + + /** + * Get the number of columns in the output schema. + * + * @return number of columns + * @throws IllegalStateException if session is closed + */ + public synchronized int getColumnCount() { + if (closed) { + throw new IllegalStateException("Streaming session has been closed"); + } + return nativeGetStreamingColumnCount(handle); + } + + /** + * Get the native handle. For use by legacy code that needs the raw handle. + * + * @return native session handle, or 0 if closed + */ + public long getHandle() { + return closed ? 0 : handle; + } + + /** + * Close the session and free native resources. + * Safe to call multiple times — subsequent calls are no-ops. + */ + @Override + public synchronized void close() { + if (!closed && handle != 0) { + nativeCloseStreamingSession(handle); + handle = 0; + closed = true; + } + } + } + + /** + * Start a streaming bulk retrieval session. + * + *

Works for both companion (parquet) and regular (tantivy doc store) splits. + * Companion splits stream from parquet files; regular splits stream from the + * tantivy document store. + * + *

Starts a background producer that streams results in batches through a + * bounded channel. Memory usage is bounded to ~24MB regardless of total result size. + * + *

Returns a {@link StreamingSession} that implements {@link AutoCloseable} + * for safe resource management. Use with try-with-resources: + * + *

{@code
+     * try (StreamingSession session = searcher.startStreamingRetrieval(queryJson, "field1")) {
+     *     int numCols = session.getColumnCount();
+     *     long[] arrayAddrs = new long[numCols];
+     *     long[] schemaAddrs = new long[numCols];
+     *     int rows;
+     *     while ((rows = session.nextBatch(arrayAddrs, schemaAddrs)) > 0) {
+     *         // process batch...
+     *     }
+     * }
+     * }
+ * + * @param queryAstJson Quickwit query AST JSON string + * @param fields field names to retrieve (null or empty for all fields) + * @return streaming session (implements AutoCloseable) + * @throws IllegalStateException if searcher is closed or query is invalid + */ + public StreamingSession startStreamingRetrieval(String queryAstJson, String... fields) { + if (nativePtr == 0) { + throw new IllegalStateException("SplitSearcher has been closed or not properly initialized"); + } + long session = nativeStartStreamingRetrieval(nativePtr, queryAstJson, + fields != null && fields.length > 0 ? fields : null); + if (session == 0) { + throw new IllegalStateException( + "Failed to start streaming retrieval — query error or no stored fields"); + } + return new StreamingSession(session); + } + + /** + * Poll the next batch from a streaming retrieval session. + * + * @deprecated Use {@link StreamingSession#nextBatch} instead for type-safe lifecycle management. + * @param sessionHandle handle from startStreamingRetrieval + * @param arrayAddrs pre-allocated ArrowArray memory addresses + * @param schemaAddrs pre-allocated ArrowSchema memory addresses + * @return row count (>0 = data, 0 = end of stream, -1 = error) + */ + @Deprecated + public int nextBatch(long sessionHandle, long[] arrayAddrs, long[] schemaAddrs) { + return nativeNextBatch(sessionHandle, arrayAddrs, schemaAddrs); + } + + /** + * Close a streaming retrieval session and free native resources. + * + *

Safe to call multiple times — the native layer uses a registry pattern + * so duplicate close calls are no-ops. + * + * @deprecated Use {@link StreamingSession#close} instead for type-safe lifecycle management. + * @param sessionHandle handle from startStreamingRetrieval + */ + @Deprecated + public void closeStreamingSession(long sessionHandle) { + if (sessionHandle != 0) { + nativeCloseStreamingSession(sessionHandle); + } + } + + /** + * Get the number of columns in a streaming session's output schema. + * + * @deprecated Use {@link StreamingSession#getColumnCount} instead. + * @param sessionHandle handle from startStreamingRetrieval + * @return number of columns, or -1 on error + */ + @Deprecated + public int getStreamingColumnCount(long sessionHandle) { + return nativeGetStreamingColumnCount(sessionHandle); + } + /** * Retrieve multiple documents by their addresses in a single batch operation. * This is significantly more efficient than calling doc() multiple times, @@ -1295,6 +1506,12 @@ private static native String nativeAggregationArrowSchema(long nativePtr, String private static native int nativeDocBatchArrowFfi(long nativePtr, int[] segments, int[] docIds, String[] fields, long[] arrayAddrs, long[] schemaAddrs); + // Streaming retrieval session (companion mode) + private static native long nativeStartStreamingRetrieval(long nativePtr, String queryAstJson, String[] fields); + private static native int nativeNextBatch(long sessionPtr, long[] arrayAddrs, long[] schemaAddrs); + private static native void nativeCloseStreamingSession(long sessionPtr); + private static native int nativeGetStreamingColumnCount(long sessionPtr); + // Smart Wildcard Optimization Statistics (for testing/monitoring) private static native void resetSmartWildcardStats(); private static native long getQueriesAnalyzed(); diff --git a/src/test/java/io/indextables/tantivy4java/LargeResultSetRetrievalTest.java b/src/test/java/io/indextables/tantivy4java/LargeResultSetRetrievalTest.java new file mode 100644 index 0000000..0c61316 --- /dev/null +++ b/src/test/java/io/indextables/tantivy4java/LargeResultSetRetrievalTest.java @@ -0,0 +1,1759 @@ +package io.indextables.tantivy4java; + +import io.indextables.tantivy4java.core.*; +import io.indextables.tantivy4java.result.SearchResult; +import io.indextables.tantivy4java.split.*; +import io.indextables.tantivy4java.split.merge.*; + +import org.junit.jupiter.api.*; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.util.*; + +import static org.junit.jupiter.api.Assertions.*; + +/** + * Integration tests for the streaming companion-mode retrieval API: + * {@code startStreamingRetrieval()} / {@code nextBatch()} / {@code closeStreamingSession()}. + * + *

These APIs bypass Quickwit's leaf_search, eliminate BM25 scoring, and resolve + * doc-to-parquet locations entirely in Rust. Companion-mode only. + * + *

Some tests also exercise the deprecated {@code searchAndRetrieveArrowFfi()} wrapper + * which now delegates to the streaming path internally. + */ +@TestMethodOrder(MethodOrderer.OrderAnnotation.class) +public class LargeResultSetRetrievalTest { + + private static SplitCacheManager cacheManager; + private static sun.misc.Unsafe unsafe; + + private static final int FFI_STRUCT_SIZE = 256; + + @BeforeAll + static void setUp() throws Exception { + SplitCacheManager.CacheConfig config = new SplitCacheManager.CacheConfig("large-result-test") + .withMaxCacheSize(100_000_000); + cacheManager = SplitCacheManager.getInstance(config); + + java.lang.reflect.Field f = sun.misc.Unsafe.class.getDeclaredField("theUnsafe"); + f.setAccessible(true); + unsafe = (sun.misc.Unsafe) f.get(null); + } + + @AfterAll + static void tearDown() { + if (cacheManager != null) { + try { cacheManager.close(); } catch (Exception e) { /* ignore */ } + } + } + + // ==================================================================== + // FFI memory helpers (same patterns as ParquetCompanionArrowFfiTest) + // ==================================================================== + + private long[][] allocateFfiStructs(int numColumns) { + long[] arrayAddrs = new long[numColumns]; + long[] schemaAddrs = new long[numColumns]; + for (int i = 0; i < numColumns; i++) { + arrayAddrs[i] = unsafe.allocateMemory(FFI_STRUCT_SIZE); + unsafe.setMemory(arrayAddrs[i], FFI_STRUCT_SIZE, (byte) 0); + schemaAddrs[i] = unsafe.allocateMemory(FFI_STRUCT_SIZE); + unsafe.setMemory(schemaAddrs[i], FFI_STRUCT_SIZE, (byte) 0); + } + return new long[][] { arrayAddrs, schemaAddrs }; + } + + private void freeFfiStructs(long[] arrayAddrs, long[] schemaAddrs) { + for (long addr : arrayAddrs) { + if (addr != 0) unsafe.freeMemory(addr); + } + for (long addr : schemaAddrs) { + if (addr != 0) unsafe.freeMemory(addr); + } + } + + private long readArrowArrayLength(long addr) { + return unsafe.getLong(addr); + } + + private long readArrowArrayRelease(long addr) { + return unsafe.getLong(addr + 64); + } + + private long readArrowSchemaRelease(long addr) { + return unsafe.getLong(addr + 56); + } + + private String readCString(long ptr) { + if (ptr == 0) return null; + StringBuilder sb = new StringBuilder(); + for (int off = 0; ; off++) { + byte b = unsafe.getByte(ptr + off); + if (b == 0) break; + sb.append((char) b); + } + return sb.toString(); + } + + private String readSchemaName(long schemaAddr) { + return readCString(unsafe.getLong(schemaAddr + 8)); + } + + private String readSchemaFormat(long schemaAddr) { + return readCString(unsafe.getLong(schemaAddr)); + } + + private long readInt64(long arrayAddr, int row) { + long dataPtr = unsafe.getLong(unsafe.getLong(arrayAddr + 40) + 8); + return unsafe.getLong(dataPtr + (long) row * 8); + } + + private double readFloat64(long arrayAddr, int row) { + long dataPtr = unsafe.getLong(unsafe.getLong(arrayAddr + 40) + 8); + return unsafe.getDouble(dataPtr + (long) row * 8); + } + + private String readUtf8(long arrayAddr, int row) { + long buffersPtr = unsafe.getLong(arrayAddr + 40); + long offsetsPtr = unsafe.getLong(buffersPtr + 8); + long dataPtr = unsafe.getLong(buffersPtr + 16); + int start = unsafe.getInt(offsetsPtr + (long) row * 4); + int end = unsafe.getInt(offsetsPtr + (long) (row + 1) * 4); + byte[] bytes = new byte[end - start]; + for (int i = 0; i < bytes.length; i++) { + bytes[i] = unsafe.getByte(dataPtr + start + i); + } + return new String(bytes, StandardCharsets.UTF_8); + } + + private boolean readBoolean(long arrayAddr, int row) { + long dataPtr = unsafe.getLong(unsafe.getLong(arrayAddr + 40) + 8); + byte b = unsafe.getByte(dataPtr + row / 8); + return (b & (1 << (row % 8))) != 0; + } + + private Map buildColumnMap(long[] schemaAddrs, int numCols) { + Map map = new HashMap<>(numCols * 2); + for (int i = 0; i < numCols; i++) { + String colName = readSchemaName(schemaAddrs[i]); + if (colName != null) map.put(colName, i); + } + return map; + } + + // ==================================================================== + // Helper: create a companion split with test parquet data + // ==================================================================== + + private static void writeTestParquet(String path, int numRows, long idOffset) { + QuickwitSplit.nativeWriteTestParquet(path, numRows, idOffset); + } + + /** + * Create a companion split with one parquet file. + * Test parquet has columns: id (i64), name (utf8), score (f64), active (bool), category (utf8). + */ + private static final int ALL_COLS = 5; + private QuickwitSplit.SplitMetadata createSingleFileSplit( + Path dir, String splitName, int numRows) throws Exception { + Path parquetFile = dir.resolve(splitName + ".parquet"); + Path splitFile = dir.resolve(splitName + ".split"); + writeTestParquet(parquetFile.toString(), numRows, 0); + + ParquetCompanionConfig config = new ParquetCompanionConfig(dir.toString()); + return QuickwitSplit.createFromParquet( + Collections.singletonList(parquetFile.toString()), + splitFile.toString(), config); + } + + /** + * Create a companion split with multiple parquet files. + */ + private QuickwitSplit.SplitMetadata createMultiFileSplit( + Path dir, String splitName, int filesCount, int rowsPerFile) throws Exception { + List parquetFiles = new ArrayList<>(); + for (int i = 0; i < filesCount; i++) { + Path pf = dir.resolve(splitName + "_part" + i + ".parquet"); + writeTestParquet(pf.toString(), rowsPerFile, (long) i * rowsPerFile); + parquetFiles.add(pf.toString()); + } + + Path splitFile = dir.resolve(splitName + ".split"); + ParquetCompanionConfig config = new ParquetCompanionConfig(dir.toString()); + return QuickwitSplit.createFromParquet(parquetFiles, splitFile.toString(), config); + } + + private SplitSearcher openSearcher(Path dir, String splitName, + QuickwitSplit.SplitMetadata metadata) throws Exception { + String splitUrl = "file://" + dir.resolve(splitName + ".split").toAbsolutePath(); + return cacheManager.createSplitSearcher(splitUrl, metadata, dir.toString()); + } + + // ==================================================================== + // STREAMING RETRIEVAL TESTS + // ==================================================================== + + @Test + @Order(10) + @DisplayName("Streaming: basic lifecycle — start, fetch batches, close") + void testStreamingBasicLifecycle(@TempDir Path dir) throws Exception { + int numRows = 100; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "stream_basic", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "stream_basic", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson); + try { + int numCols = session.getColumnCount(); + assertTrue(numCols > 0, "Should have at least 1 column"); + + int totalRows = 0; + int batchCount = 0; + while (true) { + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows == 0) break; + assertTrue(rows > 0, "nextBatch should return positive row count or 0"); + totalRows += rows; + batchCount++; + + // Verify FFI structs are valid + for (int c = 0; c < numCols; c++) { + assertEquals(rows, readArrowArrayLength(structs[0][c]), + "All columns should have same row count"); + assertNotEquals(0, readArrowArrayRelease(structs[0][c])); + } + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + + assertEquals(numRows, totalRows, + "Total streamed rows should equal input document count"); + assertTrue(batchCount >= 1, "Should have at least 1 batch"); + } finally { + session.close(); + } + } + } + + @Test + @Order(11) + @DisplayName("Streaming: column count matches projected fields") + void testStreamingColumnCount(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "stream_cols", 20); + + try (SplitSearcher searcher = openSearcher(dir, "stream_cols", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + // All fields + SplitSearcher.StreamingSession session1 = searcher.startStreamingRetrieval(queryJson); + try { + int allCols = session1.getColumnCount(); + assertTrue(allCols >= ALL_COLS, "All-field session should have >= " + ALL_COLS + " columns"); + } finally { + session1.close(); + } + + // Projected: 2 fields + SplitSearcher.StreamingSession session2 = searcher.startStreamingRetrieval(queryJson, "id", "name"); + try { + int projCols = session2.getColumnCount(); + assertEquals(2, projCols, "Projected session should have 2 columns"); + } finally { + session2.close(); + } + } + } + + @Test + @Order(12) + @DisplayName("Streaming: no matches yields empty stream") + void testStreamingNoMatches(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "stream_empty", 20); + + try (SplitSearcher searcher = openSearcher(dir, "stream_empty", metadata)) { + String queryJson = new SplitTermQuery("name", "nonexistent_xyz").toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson); + try { + int numCols = session.getColumnCount(); + assertTrue(numCols > 0, "Even empty session should report column count"); + + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + assertEquals(0, rows, "No-match query should return 0 on first nextBatch"); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } finally { + session.close(); + } + } + } + + @Test + @Order(13) + @DisplayName("Streaming: term query returns correct subset") + void testStreamingTermQuery(@TempDir Path dir) throws Exception { + int numRows = 50; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "stream_term", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "stream_term", metadata)) { + String queryJson = new SplitTermQuery("name", "item_25").toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id", "name"); + try { + int numCols = session.getColumnCount(); + assertEquals(2, numCols); + + int totalRows = 0; + List ids = new ArrayList<>(); + while (true) { + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows == 0) break; + + Map colMap = buildColumnMap(structs[1], numCols); + int idCol = colMap.get("id"); + for (int r = 0; r < rows; r++) { + ids.add(readInt64(structs[0][idCol], r)); + } + totalRows += rows; + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + + assertEquals(1, totalRows, "Term query should match exactly 1 doc"); + assertEquals(25L, ids.get(0), "Matched doc should have id=25"); + } finally { + session.close(); + } + } + } + + @Test + @Order(14) + @DisplayName("Streaming: multi-file split returns rows from all files") + void testStreamingMultiFile(@TempDir Path dir) throws Exception { + int filesCount = 3; + int rowsPerFile = 30; + int totalExpected = filesCount * rowsPerFile; + QuickwitSplit.SplitMetadata metadata = createMultiFileSplit( + dir, "stream_multi", filesCount, rowsPerFile); + + try (SplitSearcher searcher = openSearcher(dir, "stream_multi", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id"); + try { + int numCols = session.getColumnCount(); + assertEquals(1, numCols); + + Set allIds = new HashSet<>(); + while (true) { + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows == 0) break; + for (int r = 0; r < rows; r++) { + allIds.add(readInt64(structs[0][0], r)); + } + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + + assertEquals(totalExpected, allIds.size(), + "Should have unique IDs from all files"); + + // Verify IDs span the full range [0, totalExpected) + for (long i = 0; i < totalExpected; i++) { + assertTrue(allIds.contains(i), + "Should contain id=" + i + " from multi-file split"); + } + } finally { + session.close(); + } + } + } + + @Test + @Order(15) + @DisplayName("Streaming: data matches docBatch retrieval") + void testStreamingMatchesDocBatch(@TempDir Path dir) throws Exception { + int numRows = 40; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "stream_vs_batch", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "stream_vs_batch", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + // DocBatch path + SplitQuery query = searcher.parseQuery("*"); + SearchResult results = searcher.search(query, numRows); + DocAddress[] docAddrs = new DocAddress[results.getHits().size()]; + for (int i = 0; i < docAddrs.length; i++) { + docAddrs[i] = results.getHits().get(i).getDocAddress(); + } + List batchDocs = searcher.docBatchProjected(docAddrs, "id"); + Set batchIds = new HashSet<>(); + for (Document doc : batchDocs) { + batchIds.add(((Number) doc.getFirst("id")).longValue()); + } + + // Streaming path + Set streamIds = new HashSet<>(); + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id"); + try { + int streamCols = session.getColumnCount(); + while (true) { + long[][] structs = allocateFfiStructs(streamCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows == 0) break; + for (int r = 0; r < rows; r++) { + streamIds.add(readInt64(structs[0][0], r)); + } + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } finally { + session.close(); + } + + assertEquals(batchIds, streamIds, + "Streaming and docBatch should return the same document IDs"); + } + } + + @Test + @Order(16) + @DisplayName("Streaming: close session releases resources") + void testStreamingCloseReleasesResources(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "stream_dblclose", 10); + + try (SplitSearcher searcher = openSearcher(dir, "stream_dblclose", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson); + assertNotNull(session, "Session should be non-null"); + // Drain and close + int cols = session.getColumnCount(); + long[][] ffi = allocateFfiStructs(cols); + try { + while (session.nextBatch(ffi[0], ffi[1]) > 0) { + // drain + } + } finally { + freeFfiStructs(ffi[0], ffi[1]); + } + session.close(); + // Double-close is safe — native layer uses registry pattern, + // second close is a no-op (entry already removed). + session.close(); + } + } + + @Test + @Order(17) + @DisplayName("Streaming: close with handle 0 is a no-op") + void testStreamingCloseZeroHandle(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "stream_zero", 10); + + try (SplitSearcher searcher = openSearcher(dir, "stream_zero", metadata)) { + // Should not throw + searcher.closeStreamingSession(0); + } + } + + @Test + @Order(18) + @DisplayName("Streaming: verify schema field names in FFI output") + void testStreamingSchemaNames(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "stream_schema", 20); + + try (SplitSearcher searcher = openSearcher(dir, "stream_schema", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id", "score", "name"); + try { + int numCols = session.getColumnCount(); + assertEquals(3, numCols); + + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + assertTrue(rows > 0); + + Map colMap = buildColumnMap(structs[1], numCols); + assertTrue(colMap.containsKey("id"), "Should have 'id' column"); + assertTrue(colMap.containsKey("score"), "Should have 'score' column"); + assertTrue(colMap.containsKey("name"), "Should have 'name' column"); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } finally { + session.close(); + } + } + } + + @Test + @Order(19) + @DisplayName("Streaming: verify data types in Arrow format strings") + void testStreamingArrowFormatStrings(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "stream_fmt", 10); + + try (SplitSearcher searcher = openSearcher(dir, "stream_fmt", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson); + try { + int numCols = session.getColumnCount(); + + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + assertTrue(rows > 0); + + Map colMap = buildColumnMap(structs[1], numCols); + + // Verify Arrow format strings for known columns + if (colMap.containsKey("id")) { + String fmt = readSchemaFormat(structs[1][colMap.get("id")]); + assertTrue(fmt.equals("l") || fmt.equals("L"), + "id should be int64 (l) or uint64 (L), got: " + fmt); + } + if (colMap.containsKey("score")) { + String fmt = readSchemaFormat(structs[1][colMap.get("score")]); + assertEquals("g", fmt, "score should be float64 (g)"); + } + if (colMap.containsKey("name")) { + String fmt = readSchemaFormat(structs[1][colMap.get("name")]); + assertTrue(fmt.equals("u") || fmt.equals("U"), + "name should be utf8 (u) or large-utf8 (U), got: " + fmt); + } + if (colMap.containsKey("active")) { + String fmt = readSchemaFormat(structs[1][colMap.get("active")]); + assertEquals("b", fmt, "active should be boolean (b)"); + } + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } finally { + session.close(); + } + } + } + + @Test + @Order(20) + @DisplayName("Streaming: read and verify actual field values") + void testStreamingReadValues(@TempDir Path dir) throws Exception { + int numRows = 30; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "stream_values", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "stream_values", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id", "name", "score"); + try { + int numCols = session.getColumnCount(); + + Set ids = new HashSet<>(); + Set names = new HashSet<>(); + while (true) { + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows == 0) break; + + Map colMap = buildColumnMap(structs[1], numCols); + int idCol = colMap.get("id"); + int nameCol = colMap.get("name"); + int scoreCol = colMap.get("score"); + + for (int r = 0; r < rows; r++) { + long id = readInt64(structs[0][idCol], r); + ids.add(id); + + String name = readUtf8(structs[0][nameCol], r); + names.add(name); + assertTrue(name.startsWith("item_"), + "Name should start with 'item_', got: " + name); + + double score = readFloat64(structs[0][scoreCol], r); + assertTrue(score >= 0.0, + "Score should be non-negative, got: " + score); + } + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + + assertEquals(numRows, ids.size(), "Should have all unique IDs"); + assertEquals(numRows, names.size(), "Should have all unique names"); + } finally { + session.close(); + } + } + } + + // ==================================================================== + // CROSS-PATH CONSISTENCY TESTS + // ==================================================================== + + @Test + @Order(30) + @DisplayName("Cross-path: streaming and docBatch return same data") + void testAllPathsConsistent(@TempDir Path dir) throws Exception { + int numRows = 35; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "cross_path", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "cross_path", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + // Path 1: docBatch (existing API) + SplitQuery query = searcher.parseQuery("*"); + SearchResult results = searcher.search(query, numRows); + DocAddress[] docAddrs = new DocAddress[results.getHits().size()]; + for (int i = 0; i < docAddrs.length; i++) { + docAddrs[i] = results.getHits().get(i).getDocAddress(); + } + List docs = searcher.docBatchProjected(docAddrs, "id"); + Set docBatchIds = new HashSet<>(); + for (Document doc : docs) { + docBatchIds.add(((Number) doc.getFirst("id")).longValue()); + } + + // Path 2: streaming + Set streamIds = new HashSet<>(); + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id"); + try { + int numCols = session.getColumnCount(); + while (true) { + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows == 0) break; + for (int r = 0; r < rows; r++) { + streamIds.add(readInt64(structs[0][0], r)); + } + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } finally { + session.close(); + } + + assertEquals(docBatchIds, streamIds, "Streaming IDs should match docBatch IDs"); + } + } + + @Test + @Order(31) + @DisplayName("Cross-path: partial query returns consistent results across streaming and docBatch") + void testPartialQueryConsistency(@TempDir Path dir) throws Exception { + int numRows = 50; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "cross_partial", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "cross_partial", metadata)) { + String queryJson = new SplitTermQuery("name", "item_5").toQueryAstJson(); + + // Streaming + long streamId = -1; + String streamName = null; + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id", "name"); + try { + int numCols = session.getColumnCount(); + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + assertEquals(1, rows); + Map colMap = buildColumnMap(structs[1], numCols); + streamId = readInt64(structs[0][colMap.get("id")], 0); + streamName = readUtf8(structs[0][colMap.get("name")], 0); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } finally { + session.close(); + } + + assertEquals(5L, streamId, "Should be id=5"); + assertEquals("item_5", streamName, "Should be item_5"); + } + } + + // ==================================================================== + // LARGER DATASET TESTS + // ==================================================================== + + @Test + @Order(40) + @DisplayName("Streaming: larger dataset with multiple batches") + void testStreamingLargerDataset(@TempDir Path dir) throws Exception { + // 500 rows across 5 files — ensures multi-file + multi-batch behavior + int filesCount = 5; + int rowsPerFile = 100; + int totalRows = filesCount * rowsPerFile; + + QuickwitSplit.SplitMetadata metadata = createMultiFileSplit( + dir, "stream_large", filesCount, rowsPerFile); + + try (SplitSearcher searcher = openSearcher(dir, "stream_large", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id"); + try { + int numCols = session.getColumnCount(); + Set allIds = new HashSet<>(); + int batchCount = 0; + + while (true) { + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows == 0) break; + for (int r = 0; r < rows; r++) { + allIds.add(readInt64(structs[0][0], r)); + } + batchCount++; + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + + assertEquals(totalRows, allIds.size(), + "Should have " + totalRows + " unique IDs across all batches"); + assertTrue(batchCount >= 1, "Should have at least 1 batch"); + + // Verify full ID range + for (long i = 0; i < totalRows; i++) { + assertTrue(allIds.contains(i), "Missing id=" + i); + } + } finally { + session.close(); + } + } + } + + @Test + @Order(41) + @DisplayName("Fused: larger single-file dataset") + void testFusedLargerDataset(@TempDir Path dir) throws Exception { + int numRows = 200; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "fused_large", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "fused_large", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + int numCols = 2; + long[][] structs = allocateFfiStructs(numCols); + try { + int rowCount = searcher.searchAndRetrieveArrowFfi( + queryJson, structs[0], structs[1], "id", "name"); + assertEquals(numRows, rowCount); + + Map colMap = buildColumnMap(structs[1], numCols); + Set ids = new HashSet<>(); + for (int r = 0; r < rowCount; r++) { + ids.add(readInt64(structs[0][colMap.get("id")], r)); + } + assertEquals(numRows, ids.size(), "All IDs should be unique"); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } + + // ==================================================================== + // ERROR HANDLING TESTS + // ==================================================================== + + @Test + @Order(50) + @DisplayName("Error: fused with non-companion split returns -1") + void testFusedNonCompanionSplit(@TempDir Path dir) throws Exception { + // Create a regular (non-companion) Quickwit split from a Tantivy index + // Now that streaming works for non-companion splits, the deprecated wrapper should succeed + try (SchemaBuilder builder = new SchemaBuilder()) { + builder.addTextField("title", true, false, "default", "position"); + builder.addIntegerField("count", true, true, false); + try (Schema schema = builder.build(); + Index index = new Index(schema, dir.resolve("idx").toString(), false); + IndexWriter writer = index.writer(Index.Memory.DEFAULT_HEAP_SIZE, 1)) { + try (Document doc = new Document()) { + doc.addText("title", "hello world"); + doc.addInteger("count", 42); + writer.addDocument(doc); + } + writer.commit(); + } + } + QuickwitSplit.SplitConfig splitConfig = new QuickwitSplit.SplitConfig( + "test-idx", "test-src", "test-node"); + QuickwitSplit.SplitMetadata metadata = QuickwitSplit.convertIndexFromPath( + dir.resolve("idx").toString(), + dir.resolve("regular.split").toString(), splitConfig); + + String splitUrl = "file://" + dir.resolve("regular.split").toAbsolutePath(); + try (SplitSearcher searcher = cacheManager.createSplitSearcher(splitUrl, metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + long[][] structs = allocateFfiStructs(2); + try { + int rowCount = searcher.searchAndRetrieveArrowFfi( + queryJson, structs[0], structs[1], "title"); + assertTrue(rowCount >= 1, + "Non-companion split streaming should return rows, got " + rowCount); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } + + @Test + @Order(51) + @DisplayName("Streaming with non-companion split works via tantivy doc store") + void testStreamingNonCompanionSplit(@TempDir Path dir) throws Exception { + try (SchemaBuilder builder = new SchemaBuilder()) { + builder.addTextField("title", true, false, "default", "position"); + try (Schema schema = builder.build(); + Index index = new Index(schema, dir.resolve("idx2").toString(), false); + IndexWriter writer = index.writer(Index.Memory.DEFAULT_HEAP_SIZE, 1)) { + try (Document doc = new Document()) { + doc.addText("title", "test"); + writer.addDocument(doc); + } + writer.commit(); + } + } + QuickwitSplit.SplitConfig splitConfig = new QuickwitSplit.SplitConfig( + "test-idx", "test-src", "test-node"); + QuickwitSplit.SplitMetadata metadata = QuickwitSplit.convertIndexFromPath( + dir.resolve("idx2").toString(), + dir.resolve("regular2.split").toString(), splitConfig); + + String splitUrl = "file://" + dir.resolve("regular2.split").toAbsolutePath(); + try (SplitSearcher searcher = cacheManager.createSplitSearcher(splitUrl, metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + try (SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson)) { + int numCols = session.getColumnCount(); + assertTrue(numCols >= 1, "Should have at least 1 column"); + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + assertTrue(rows >= 1, "Should retrieve at least 1 row from non-companion split"); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } + } + + @Test + @Order(52) + @DisplayName("Error: malformed query JSON throws RuntimeException") + void testMalformedQueryJson(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "err_malformed", 10); + + try (SplitSearcher searcher = openSearcher(dir, "err_malformed", metadata)) { + long[][] structs = allocateFfiStructs(ALL_COLS); + try { + assertThrows(RuntimeException.class, () -> { + searcher.searchAndRetrieveArrowFfi( + "{invalid json!!!}", structs[0], structs[1]); + }, "Malformed JSON should throw RuntimeException"); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } + + @Test + @Order(53) + @DisplayName("Error: malformed query JSON in streaming throws exception") + void testMalformedQueryJsonStreaming(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "err_malformed_s", 10); + + try (SplitSearcher searcher = openSearcher(dir, "err_malformed_s", metadata)) { + assertThrows(Exception.class, () -> { + searcher.startStreamingRetrieval("{not valid json"); + }, "Malformed JSON should throw on streaming start"); + } + } + + @Test + @Order(54) + @DisplayName("Error: fused returns -1 for non-companion after close") + void testFusedAfterSearcherClose(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "err_closed", 10); + SplitSearcher searcher = openSearcher(dir, "err_closed", metadata); + searcher.close(); + + assertThrows(IllegalStateException.class, () -> { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + long[][] structs = allocateFfiStructs(ALL_COLS); + searcher.searchAndRetrieveArrowFfi(queryJson, structs[0], structs[1]); + }, "Closed searcher should throw IllegalStateException"); + } + + // ==================================================================== + // QUERY TYPE TESTS (Boolean, Range, Exists, Wildcard) + // ==================================================================== + + @Test + @Order(60) + @DisplayName("Query: boolean AND query via fused path") + void testBooleanAndQuery(@TempDir Path dir) throws Exception { + int numRows = 50; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "q_bool_and", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "q_bool_and", metadata)) { + // Match items with name=item_10 AND category must match + SplitBooleanQuery boolQuery = new SplitBooleanQuery(); + boolQuery.addMust(new SplitTermQuery("name", "item_10")); + boolQuery.addMust(new SplitMatchAllQuery()); + String queryJson = boolQuery.toQueryAstJson(); + + long[][] structs = allocateFfiStructs(2); + try { + int rowCount = searcher.searchAndRetrieveArrowFfi( + queryJson, structs[0], structs[1], "id", "name"); + assertEquals(1, rowCount, "Boolean AND should narrow to 1 result"); + + Map colMap = buildColumnMap(structs[1], 2); + long id = readInt64(structs[0][colMap.get("id")], 0); + assertEquals(10L, id); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } + + @Test + @Order(61) + @DisplayName("Query: boolean OR query via streaming path") + void testBooleanOrQueryStreaming(@TempDir Path dir) throws Exception { + int numRows = 50; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "q_bool_or", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "q_bool_or", metadata)) { + SplitBooleanQuery boolQuery = new SplitBooleanQuery(); + boolQuery.addShould(new SplitTermQuery("name", "item_5")); + boolQuery.addShould(new SplitTermQuery("name", "item_15")); + boolQuery.addShould(new SplitTermQuery("name", "item_25")); + boolQuery.setMinimumShouldMatch(1); + String queryJson = boolQuery.toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id"); + try { + int numCols = session.getColumnCount(); + Set ids = new HashSet<>(); + while (true) { + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows == 0) break; + for (int r = 0; r < rows; r++) { + ids.add(readInt64(structs[0][0], r)); + } + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + assertEquals(3, ids.size(), "Boolean OR should match 3 items"); + assertTrue(ids.contains(5L)); + assertTrue(ids.contains(15L)); + assertTrue(ids.contains(25L)); + } finally { + session.close(); + } + } + } + + @Test + @Order(62) + @DisplayName("Query: boolean MUST_NOT excludes results") + void testBooleanMustNotQuery(@TempDir Path dir) throws Exception { + int numRows = 10; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "q_bool_not", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "q_bool_not", metadata)) { + SplitBooleanQuery boolQuery = new SplitBooleanQuery(); + boolQuery.addMust(new SplitMatchAllQuery()); + boolQuery.addMustNot(new SplitTermQuery("name", "item_5")); + String queryJson = boolQuery.toQueryAstJson(); + + long[][] structs = allocateFfiStructs(1); + try { + int rowCount = searcher.searchAndRetrieveArrowFfi( + queryJson, structs[0], structs[1], "id"); + assertEquals(numRows - 1, rowCount, + "MUST_NOT should exclude 1 row"); + + Set ids = new HashSet<>(); + for (int r = 0; r < rowCount; r++) { + ids.add(readInt64(structs[0][0], r)); + } + assertFalse(ids.contains(5L), "id=5 should be excluded"); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } + + @Test + @Order(63) + @DisplayName("Query: range query on numeric field") + void testRangeQuery(@TempDir Path dir) throws Exception { + int numRows = 50; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "q_range", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "q_range", metadata)) { + // id field is i64, range [10, 20] inclusive + SplitRangeQuery rangeQuery = SplitRangeQuery.inclusiveRange( + "id", "10", "20", "i64"); + String queryJson = rangeQuery.toQueryAstJson(); + + long[][] structs = allocateFfiStructs(1); + try { + int rowCount = searcher.searchAndRetrieveArrowFfi( + queryJson, structs[0], structs[1], "id"); + assertEquals(11, rowCount, "Range [10,20] inclusive should match 11 rows"); + + Set ids = new HashSet<>(); + for (int r = 0; r < rowCount; r++) { + ids.add(readInt64(structs[0][0], r)); + } + for (long i = 10; i <= 20; i++) { + assertTrue(ids.contains(i), "Should contain id=" + i); + } + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } + + @Test + @Order(64) + @DisplayName("Query: exists query on field") + void testExistsQuery(@TempDir Path dir) throws Exception { + int numRows = 20; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "q_exists", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "q_exists", metadata)) { + SplitExistsQuery existsQuery = new SplitExistsQuery("name"); + String queryJson = existsQuery.toQueryAstJson(); + + long[][] structs = allocateFfiStructs(1); + try { + int rowCount = searcher.searchAndRetrieveArrowFfi( + queryJson, structs[0], structs[1], "id"); + assertEquals(numRows, rowCount, + "All rows should have the 'name' field"); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } + + @Test + @Order(65) + @DisplayName("Query: range query via streaming path") + void testRangeQueryStreaming(@TempDir Path dir) throws Exception { + int numRows = 100; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "q_range_s", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "q_range_s", metadata)) { + // Exclusive range (5, 15) should match 5 < id < 15 → ids 6..14 = 9 rows + SplitRangeQuery rangeQuery = SplitRangeQuery.exclusiveRange( + "id", "5", "15", "i64"); + String queryJson = rangeQuery.toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id"); + try { + int numCols = session.getColumnCount(); + Set ids = new HashSet<>(); + while (true) { + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows == 0) break; + for (int r = 0; r < rows; r++) { + ids.add(readInt64(structs[0][0], r)); + } + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + assertEquals(9, ids.size(), "Exclusive range (5,15) should match 9 rows"); + assertFalse(ids.contains(5L), "5 should be excluded (exclusive)"); + assertFalse(ids.contains(15L), "15 should be excluded (exclusive)"); + assertTrue(ids.contains(10L), "10 should be included"); + } finally { + session.close(); + } + } + } + + // ==================================================================== + // SINGLE ROW AND EDGE CASE TESTS + // ==================================================================== + + @Test + @Order(70) + @DisplayName("Edge: single row result via streaming") + void testStreamingSingleRow(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "edge_single", 50); + + try (SplitSearcher searcher = openSearcher(dir, "edge_single", metadata)) { + String queryJson = new SplitTermQuery("name", "item_25").toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id", "name"); + try { + int numCols = session.getColumnCount(); + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + assertEquals(1, rows, "Should get exactly 1 row"); + Map colMap = buildColumnMap(structs[1], numCols); + assertEquals(25L, readInt64(structs[0][colMap.get("id")], 0)); + assertEquals("item_25", readUtf8(structs[0][colMap.get("name")], 0)); + + // Next batch should be end of stream + long[][] structs2 = allocateFfiStructs(numCols); + try { + int rows2 = session.nextBatch(structs2[0], structs2[1]); + assertEquals(0, rows2, "Should be end of stream after single row"); + } finally { + freeFfiStructs(structs2[0], structs2[1]); + } + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } finally { + session.close(); + } + } + } + + @Test + @Order(71) + @DisplayName("Edge: single row split (1 total row)") + void testSingleRowSplit(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "edge_1row", 1); + + try (SplitSearcher searcher = openSearcher(dir, "edge_1row", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + // Fused + long[][] structs = allocateFfiStructs(ALL_COLS); + try { + int rowCount = searcher.searchAndRetrieveArrowFfi( + queryJson, structs[0], structs[1]); + assertEquals(1, rowCount, "Single-row split should return 1 row"); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + + // Streaming + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id"); + try { + int numCols = session.getColumnCount(); + long[][] s = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(s[0], s[1]); + assertEquals(1, rows, "Streaming single-row should return 1"); + } finally { + freeFfiStructs(s[0], s[1]); + } + } finally { + session.close(); + } + } + } + + @Test + @Order(72) + @DisplayName("Edge: match-all via streaming (not just fused)") + void testStreamingMatchAll(@TempDir Path dir) throws Exception { + int numRows = 75; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "edge_matchall_s", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "edge_matchall_s", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id"); + try { + int numCols = session.getColumnCount(); + int totalRows = 0; + while (true) { + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows == 0) break; + totalRows += rows; + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + assertEquals(numRows, totalRows, "Streaming match-all should return all rows"); + } finally { + session.close(); + } + } + } + + @Test + @Order(73) + @DisplayName("Edge: query matching zero docs in some files but docs in others") + void testPartialFileMatch(@TempDir Path dir) throws Exception { + // File 0: ids 0-19 (names item_0..item_19) + // File 1: ids 20-39 (names item_20..item_39) + // Query for item_25 should only match in file 1 + QuickwitSplit.SplitMetadata metadata = createMultiFileSplit(dir, "edge_partial", 2, 20); + + try (SplitSearcher searcher = openSearcher(dir, "edge_partial", metadata)) { + String queryJson = new SplitTermQuery("name", "item_25").toQueryAstJson(); + + long[][] structs = allocateFfiStructs(1); + try { + int rowCount = searcher.searchAndRetrieveArrowFfi( + queryJson, structs[0], structs[1], "id"); + assertEquals(1, rowCount, "Should match exactly 1 row from file 1"); + assertEquals(25L, readInt64(structs[0][0], 0)); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } + + // ==================================================================== + // CONCURRENT STREAMING SESSIONS + // ==================================================================== + + @Test + @Order(80) + @DisplayName("Concurrent: multiple streaming sessions simultaneously") + void testConcurrentStreamingSessions(@TempDir Path dir) throws Exception { + int numRows = 30; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "concurrent", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "concurrent", metadata)) { + // Start two sessions with different queries + String allQuery = new SplitMatchAllQuery().toQueryAstJson(); + String termQuery = new SplitTermQuery("name", "item_10").toQueryAstJson(); + + SplitSearcher.StreamingSession session1 = searcher.startStreamingRetrieval(allQuery, "id"); + SplitSearcher.StreamingSession session2 = searcher.startStreamingRetrieval(termQuery, "id"); + try { + // Drain session 1 + int numCols1 = session1.getColumnCount(); + Set ids1 = new HashSet<>(); + while (true) { + long[][] s = allocateFfiStructs(numCols1); + try { + int rows = session1.nextBatch(s[0], s[1]); + if (rows == 0) break; + for (int r = 0; r < rows; r++) ids1.add(readInt64(s[0][0], r)); + } finally { + freeFfiStructs(s[0], s[1]); + } + } + + // Drain session 2 + int numCols2 = session2.getColumnCount(); + Set ids2 = new HashSet<>(); + while (true) { + long[][] s = allocateFfiStructs(numCols2); + try { + int rows = session2.nextBatch(s[0], s[1]); + if (rows == 0) break; + for (int r = 0; r < rows; r++) ids2.add(readInt64(s[0][0], r)); + } finally { + freeFfiStructs(s[0], s[1]); + } + } + + assertEquals(numRows, ids1.size(), "Session 1 should have all rows"); + assertEquals(1, ids2.size(), "Session 2 should have 1 row"); + assertTrue(ids2.contains(10L)); + } finally { + session1.close(); + session2.close(); + } + } + } + + @Test + @Order(81) + @DisplayName("Concurrent: interleaved nextBatch calls from two sessions") + void testInterleavedNextBatch(@TempDir Path dir) throws Exception { + int numRows = 50; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "interleaved", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "interleaved", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + SplitSearcher.StreamingSession session1 = searcher.startStreamingRetrieval(queryJson, "id"); + SplitSearcher.StreamingSession session2 = searcher.startStreamingRetrieval(queryJson, "id"); + try { + int cols1 = session1.getColumnCount(); + int cols2 = session2.getColumnCount(); + Set ids1 = new HashSet<>(); + Set ids2 = new HashSet<>(); + + // Interleave: pull from session1 then session2 + boolean s1done = false, s2done = false; + while (!s1done || !s2done) { + if (!s1done) { + long[][] s = allocateFfiStructs(cols1); + try { + int rows = session1.nextBatch(s[0], s[1]); + if (rows == 0) { s1done = true; } + else { for (int r = 0; r < rows; r++) ids1.add(readInt64(s[0][0], r)); } + } finally { freeFfiStructs(s[0], s[1]); } + } + if (!s2done) { + long[][] s = allocateFfiStructs(cols2); + try { + int rows = session2.nextBatch(s[0], s[1]); + if (rows == 0) { s2done = true; } + else { for (int r = 0; r < rows; r++) ids2.add(readInt64(s[0][0], r)); } + } finally { freeFfiStructs(s[0], s[1]); } + } + } + + assertEquals(numRows, ids1.size(), "Session 1 complete"); + assertEquals(numRows, ids2.size(), "Session 2 complete"); + assertEquals(ids1, ids2, "Both sessions should return same IDs"); + } finally { + session1.close(); + session2.close(); + } + } + } + + // ==================================================================== + // FIELD PROJECTION EDGE CASES + // ==================================================================== + + @Test + @Order(85) + @DisplayName("Projection: single numeric field only") + void testProjectSingleNumericField(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "proj_num", 20); + + try (SplitSearcher searcher = openSearcher(dir, "proj_num", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + long[][] structs = allocateFfiStructs(1); + try { + int rowCount = searcher.searchAndRetrieveArrowFfi( + queryJson, structs[0], structs[1], "score"); + assertEquals(20, rowCount); + String fmt = readSchemaFormat(structs[1][0]); + assertEquals("g", fmt, "score should be float64 (g)"); + // Verify first value + double v = readFloat64(structs[0][0], 0); + assertTrue(v >= 0.0, "Score should be non-negative"); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } + + @Test + @Order(86) + @DisplayName("Projection: single boolean field only") + void testProjectSingleBooleanField(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "proj_bool", 10); + + try (SplitSearcher searcher = openSearcher(dir, "proj_bool", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + long[][] structs = allocateFfiStructs(1); + try { + int rowCount = searcher.searchAndRetrieveArrowFfi( + queryJson, structs[0], structs[1], "active"); + assertEquals(10, rowCount); + String fmt = readSchemaFormat(structs[1][0]); + assertEquals("b", fmt, "active should be boolean (b)"); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + } + + @Test + @Order(87) + @DisplayName("Projection: streaming with all fields (no projection)") + void testStreamingAllFields(@TempDir Path dir) throws Exception { + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "proj_all_s", 15); + + try (SplitSearcher searcher = openSearcher(dir, "proj_all_s", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + // No field args = all fields + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson); + try { + int numCols = session.getColumnCount(); + assertTrue(numCols >= ALL_COLS, + "All-fields streaming should have >= " + ALL_COLS + " columns, got " + numCols); + + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + assertTrue(rows > 0); + // All columns should have same row count + for (int c = 0; c < numCols; c++) { + assertEquals(rows, readArrowArrayLength(structs[0][c]), + "Column " + c + " should have same row count"); + } + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } finally { + session.close(); + } + } + } + + // ==================================================================== + // CROSS-PATH CONSISTENCY WITH MULTIPLE DATA TYPES + // ==================================================================== + + @Test + @Order(90) + @DisplayName("Cross-path: boolean and float values consistent via streaming") + void testCrossPathAllTypes(@TempDir Path dir) throws Exception { + int numRows = 20; + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "cross_types", numRows); + + try (SplitSearcher searcher = openSearcher(dir, "cross_types", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + String[] fields = {"id", "score", "active"}; + + // Streaming path: collect all values + Map streamScores = new HashMap<>(); + Map streamActive = new HashMap<>(); + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, fields); + try { + int sCols = session.getColumnCount(); + while (true) { + long[][] s = allocateFfiStructs(sCols); + try { + int rows = session.nextBatch(s[0], s[1]); + if (rows == 0) break; + Map colMap = buildColumnMap(s[1], sCols); + int idCol = colMap.get("id"); + int scoreCol = colMap.get("score"); + int activeCol = colMap.get("active"); + for (int r = 0; r < rows; r++) { + long id = readInt64(s[0][idCol], r); + streamScores.put(id, readFloat64(s[0][scoreCol], r)); + streamActive.put(id, readBoolean(s[0][activeCol], r)); + } + } finally { + freeFfiStructs(s[0], s[1]); + } + } + } finally { + session.close(); + } + + // Verify we got all rows with valid values + assertEquals(numRows, streamScores.size(), "Should have all rows"); + for (Long id : streamScores.keySet()) { + assertNotNull(streamScores.get(id), "Score should not be null for id=" + id); + assertNotNull(streamActive.get(id), "Active should not be null for id=" + id); + } + } + } + + // ==================================================================== + // LARGER SCALE TESTS + // ==================================================================== + + @Test + @Order(95) + @DisplayName("Scale: 1000 rows multi-file with range query selectivity") + void testLargerScaleRangeQuery(@TempDir Path dir) throws Exception { + // 1000 rows across 10 files — tests multi-file with selective query + int filesCount = 10; + int rowsPerFile = 100; + int totalRows = filesCount * rowsPerFile; + QuickwitSplit.SplitMetadata metadata = createMultiFileSplit( + dir, "scale_range", filesCount, rowsPerFile); + + try (SplitSearcher searcher = openSearcher(dir, "scale_range", metadata)) { + // Select ~10% of rows via range query + SplitRangeQuery rangeQuery = SplitRangeQuery.inclusiveRange( + "id", "0", "99", "i64"); + String queryJson = rangeQuery.toQueryAstJson(); + + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id"); + try { + int numCols = session.getColumnCount(); + Set ids = new HashSet<>(); + int batchCount = 0; + while (true) { + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows == 0) break; + for (int r = 0; r < rows; r++) { + ids.add(readInt64(structs[0][0], r)); + } + batchCount++; + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + assertEquals(100, ids.size(), "Range [0,99] should match 100 rows"); + assertTrue(batchCount >= 1); + } finally { + session.close(); + } + } + } + + @Test + @Order(96) + @DisplayName("Scale: high selectivity (>50%) match-all multi-file") + void testHighSelectivity(@TempDir Path dir) throws Exception { + // All rows selected from multiple files — exercises FullRowGroup strategy + int filesCount = 5; + int rowsPerFile = 200; + int totalRows = filesCount * rowsPerFile; + QuickwitSplit.SplitMetadata metadata = createMultiFileSplit( + dir, "scale_high_sel", filesCount, rowsPerFile); + + try (SplitSearcher searcher = openSearcher(dir, "scale_high_sel", metadata)) { + String queryJson = new SplitMatchAllQuery().toQueryAstJson(); + + // Fused: verify all rows + long[][] structs = allocateFfiStructs(1); + try { + int rowCount = searcher.searchAndRetrieveArrowFfi( + queryJson, structs[0], structs[1], "id"); + assertEquals(totalRows, rowCount, "All " + totalRows + " rows should be returned"); + } finally { + freeFfiStructs(structs[0], structs[1]); + } + + // Streaming: verify same count + SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson, "id"); + try { + int numCols = session.getColumnCount(); + int streamTotal = 0; + while (true) { + long[][] s = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(s[0], s[1]); + if (rows == 0) break; + streamTotal += rows; + } finally { + freeFfiStructs(s[0], s[1]); + } + } + assertEquals(totalRows, streamTotal); + } finally { + session.close(); + } + } + } + + // ==================================================================== + // STREAMING QUERY REWRITE REGRESSION TESTS + // + // These tests verify that perform_bulk_search() correctly rewrites + // queries for companion splits with string hash fields and exact_only + // string indexing modes. Without these rewrites, IS NOT NULL, IS NULL, + // and exact_only EqualTo queries return incorrect results. + // + // Bug: TANTIVY4JAVA_STREAMING_FAST_FIELD_BUG + // ==================================================================== + + /** + * Helper: create a companion split from the string-indexing test parquet, + * with the given tokenizer overrides and string hash optimization enabled. + */ + private SplitSearcher createStringIndexingSplitSearcher( + Path dir, Map tokenizerOverrides, String tag) throws Exception { + Path parquetFile = dir.resolve(tag + ".parquet"); + Path splitFile = dir.resolve(tag + ".split"); + + QuickwitSplit.nativeWriteTestParquetForStringIndexing( + parquetFile.toString(), 15, 0); + + ParquetCompanionConfig config = new ParquetCompanionConfig(dir.toString()) + .withStringHashOptimization(true) + .withTokenizerOverrides(tokenizerOverrides); + + QuickwitSplit.SplitMetadata metadata = QuickwitSplit.createFromParquet( + Collections.singletonList(parquetFile.toString()), + splitFile.toString(), config); + + String splitUrl = "file://" + splitFile.toAbsolutePath(); + return cacheManager.createSplitSearcher(splitUrl, metadata, dir.toString()); + } + + /** Drain a streaming session and return total row count. */ + private int drainStreamingSession(SplitSearcher.StreamingSession session) { + int totalRows = 0; + int numCols = session.getColumnCount(); + while (true) { + long[][] structs = allocateFfiStructs(numCols); + try { + int rows = session.nextBatch(structs[0], structs[1]); + if (rows <= 0) break; + totalRows += rows; + } finally { + freeFfiStructs(structs[0], structs[1]); + } + } + return totalRows; + } + + @Test + @Order(100) + @DisplayName("Streaming: IS NOT NULL (FieldPresence) on companion split returns all rows") + void testStreamingFieldPresenceIsNotNull(@TempDir Path dir) throws Exception { + // Create companion split with string hash optimization (enables _phash_* fields) + Map overrides = Collections.emptyMap(); + try (SplitSearcher searcher = createStringIndexingSplitSearcher(dir, overrides, "fp_notnull")) { + // IS NOT NULL on trace_id — all 15 rows have this field + SplitExistsQuery existsQuery = new SplitExistsQuery("trace_id"); + String queryJson = existsQuery.toQueryAstJson(); + + // Verify via regular search path (baseline) + SearchResult baseline = searcher.search(existsQuery, 100); + assertEquals(15, baseline.getHits().size(), + "Baseline search should find all 15 rows for IS NOT NULL on trace_id"); + + // Verify via streaming path (the regression) + try (SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson)) { + int streamRows = drainStreamingSession(session); + assertEquals(15, streamRows, + "Streaming IS NOT NULL should return 15 rows, same as baseline"); + } + } + } + + @Test + @Order(101) + @DisplayName("Streaming: IS NULL (negated FieldPresence) on companion split returns 0 rows") + void testStreamingFieldPresenceIsNull(@TempDir Path dir) throws Exception { + Map overrides = Collections.emptyMap(); + try (SplitSearcher searcher = createStringIndexingSplitSearcher(dir, overrides, "fp_null")) { + // IS NULL = match_all AND NOT exists(trace_id) + // When all rows have trace_id, IS NULL should return 0 + SplitBooleanQuery isNullQuery = new SplitBooleanQuery() + .addMust(new SplitMatchAllQuery()) + .addMustNot(new SplitExistsQuery("trace_id")); + String queryJson = isNullQuery.toQueryAstJson(); + + // Verify via regular search path (baseline) + SearchResult baseline = searcher.search(isNullQuery, 100); + assertEquals(0, baseline.getHits().size(), + "Baseline IS NULL should return 0 rows when all docs have trace_id"); + + // Verify via streaming path (the regression) + try (SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson)) { + int streamRows = drainStreamingSession(session); + assertEquals(0, streamRows, + "Streaming IS NULL should return 0 rows, same as baseline"); + } + } + } + + @Test + @Order(102) + @DisplayName("Streaming: exact_only term query finds matching document") + void testStreamingExactOnlyTermQuery(@TempDir Path dir) throws Exception { + Map overrides = new HashMap<>(); + overrides.put("trace_id", ParquetCompanionConfig.StringIndexingMode.EXACT_ONLY); + + try (SplitSearcher searcher = createStringIndexingSplitSearcher(dir, overrides, "eo_stream")) { + // First, get a trace_id value from the data via regular search + SearchResult allDocs = searcher.search(new SplitMatchAllQuery(), 1); + assertEquals(1, allDocs.getHits().size()); + + String traceId; + try (Document doc = searcher.docProjected(allDocs.getHits().get(0).getDocAddress())) { + traceId = (String) doc.getFirst("trace_id"); + assertNotNull(traceId, "trace_id should be retrievable from parquet"); + } + + // Build term query for this trace_id (will be rewritten to _phash_trace_id) + SplitTermQuery termQuery = new SplitTermQuery("trace_id", traceId); + String queryJson = termQuery.toQueryAstJson(); + + // Verify via regular search path (baseline) + SearchResult baseline = searcher.search(termQuery, 10); + assertEquals(1, baseline.getHits().size(), + "Baseline exact_only term query should find exactly 1 doc"); + + // Verify via streaming path (the regression) + try (SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson)) { + int streamRows = drainStreamingSession(session); + assertEquals(1, streamRows, + "Streaming exact_only term query should find 1 row, same as baseline"); + } + } + } + + @Test + @Order(103) + @DisplayName("Streaming: exact_only EqualTo for non-existent value returns 0") + void testStreamingExactOnlyNoMatch(@TempDir Path dir) throws Exception { + Map overrides = new HashMap<>(); + overrides.put("trace_id", ParquetCompanionConfig.StringIndexingMode.EXACT_ONLY); + + try (SplitSearcher searcher = createStringIndexingSplitSearcher(dir, overrides, "eo_nomatch")) { + // Term query for a value that doesn't exist + SplitTermQuery termQuery = new SplitTermQuery("trace_id", "00000000-0000-0000-0000-000000000000"); + String queryJson = termQuery.toQueryAstJson(); + + // Verify streaming returns 0 + try (SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson)) { + int streamRows = drainStreamingSession(session); + assertEquals(0, streamRows, + "Streaming exact_only term query for non-existent value should return 0"); + } + } + } + + @Test + @Order(104) + @DisplayName("Streaming vs search() consistency for IS NOT NULL on companion split") + void testStreamingVsSearchFieldPresenceConsistency(@TempDir Path dir) throws Exception { + // Test with a basic companion split (from createSingleFileSplit) to ensure + // field presence works on the standard test parquet columns too + QuickwitSplit.SplitMetadata metadata = createSingleFileSplit(dir, "fp_consist", 50); + + try (SplitSearcher searcher = openSearcher(dir, "fp_consist", metadata)) { + // IS NOT NULL on "name" — all rows should have this column + SplitExistsQuery existsQuery = new SplitExistsQuery("name"); + String queryJson = existsQuery.toQueryAstJson(); + + SearchResult baseline = searcher.search(existsQuery, 100); + int baselineCount = (int) baseline.getHits().size(); + assertTrue(baselineCount > 0, "Baseline should find rows with name field"); + + try (SplitSearcher.StreamingSession session = searcher.startStreamingRetrieval(queryJson)) { + int streamRows = drainStreamingSession(session); + assertEquals(baselineCount, streamRows, + "Streaming IS NOT NULL row count should match search() baseline"); + } + } + } +}