From 93aae229d14fa11873783a7eee1bba0c5dceb92d Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 2 Jun 2026 18:15:34 -0500 Subject: [PATCH 01/12] test: add failing repros for active-memtable predicate-crossing stale reads The active memtable's vector (HNSW) and FTS (inverted) indexes are append-only, and both index arms dedup results with WithinSourceDedupExec, which only suppresses a stale row when the fresh version is also present in the result set. When a PK is updated out of the query's match set (vector: moved out of the neighborhood; FTS: new text no longer matches the query), the fresh row isn't returned, so the stale version leaks. Two #[ignore]d repros capture the gap: - vector_search: a PK moved far from the query still surfaces at its stale near distance because the fresh node is evicted from the over-fetched top-k. - fts_search: a PK updated "alpha lance" -> "beta lance" still matches an "alpha" query because the old posting stays live in the append-only index. Both are #[ignore]d so the default suite stays green; un-ignore once the active arms gain a predicate-independent recency filter over the whole memtable. Co-Authored-By: Claude Opus 4.8 (1M context) --- .../src/dataset/mem_wal/scanner/fts_search.rs | 93 ++++++++++++++ .../dataset/mem_wal/scanner/vector_search.rs | 114 ++++++++++++++++++ 2 files changed, 207 insertions(+) diff --git a/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs b/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs index aa086a80e66..7ff800edcc8 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs @@ -561,4 +561,97 @@ mod tests { } } } + + #[tokio::test] + #[ignore = "known bug: active-memtable predicate-crossing stale read in the FTS arm. \ + A PK updated so its new text no longer matches the query still leaks its stale \ + version, because the append-only inverted index keeps the old posting and the \ + fresh row isn't a candidate to dedup against. Un-ignore once the active arm gains \ + a predicate-independent recency filter over the whole memtable."] + async fn active_stale_update_predicate_crossing_leaks() { + // BUG REPRODUCTION (FTS case: a PK update that crosses out of the match set). + // + // pk=1 is inserted as "alpha lance", then updated to "beta lance". The + // active inverted index is append-only, so the old "alpha" posting stays + // live. A search for "alpha" still matches the STALE pk=1 row. The fresh + // row ("beta lance") doesn't contain "alpha", so it's not even a + // candidate — there's nothing in the result set to dedup the stale hit + // against. The active FTS arm has no recency filter, so pk=1 leaks. + // + // Desired (NewestPkFilterExec) behaviour: pk=1's newest row-position is + // the "beta lance" row, so the stale "alpha" hit is dropped predicate- + // independently and pk=1 must NOT be returned for the "alpha" query. + let schema = fts_schema(); + let batch_store = Arc::new(BatchStore::with_capacity(16)); + let mut indexes = IndexStore::new(); + indexes.add_fts("text_fts".to_string(), 1, "text".to_string()); + + // Insert pk=1 ("alpha lance") and an unrelated live pk=2 ("alpha foo"). + let b1 = make_batch(&schema, &[1, 2], &["alpha lance", "alpha foo"]); + let (_, _, bp1) = batch_store.append(b1.clone()).unwrap(); + indexes + .insert_with_batch_position(&b1, 0, Some(bp1)) + .unwrap(); + + // Update pk=1 → "beta lance" (no longer matches "alpha"). + let b2 = make_batch(&schema, &[1], &["beta lance"]); + let (_, _, bp2) = batch_store.append(b2.clone()).unwrap(); + indexes + .insert_with_batch_position(&b2, 1, Some(bp2)) + .unwrap(); + let indexes = Arc::new(indexes); + + let tmp = tempfile::tempdir().unwrap(); + let base_uri = format!("{}/base", tmp.path().to_str().unwrap()); + let collector = LsmDataSourceCollector::without_base_table(base_uri, vec![]) + .with_in_memory_memtables( + uuid::Uuid::new_v4(), + InMemoryMemTables { + active: InMemoryMemTableRef { + batch_store, + index_store: indexes, + schema: schema.clone(), + generation: 1, + }, + frozen: vec![], + }, + ); + + let planner = LsmFtsSearchPlanner::new(collector, vec!["id".to_string()], schema); + let plan = planner + .plan_search( + "text", + FullTextSearchQuery::new("alpha".to_string()), + 10, + None, + ) + .await + .expect("planner should produce a plan"); + + let ctx = datafusion::prelude::SessionContext::new(); + let stream = plan.execute(0, ctx.task_ctx()).unwrap(); + let batches: Vec = stream.try_collect().await.unwrap(); + + let mut ids: Vec = Vec::new(); + for b in &batches { + let col = b + .column_by_name("id") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..b.num_rows() { + ids.push(col.value(i)); + } + } + + assert!( + !ids.contains(&1), + "stale pk=1 (now 'beta lance') leaked on an 'alpha' search; got ids={ids:?}" + ); + assert!( + ids.contains(&2), + "live pk=2 ('alpha foo') must still match 'alpha'; got ids={ids:?}" + ); + } } diff --git a/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs b/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs index b6b1f952b25..a6a6ddb3538 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs @@ -1549,6 +1549,120 @@ mod tests { ); } + #[tokio::test] + #[ignore = "known bug: active-memtable predicate-crossing stale read in the vector arm. \ + A PK updated out of the query's neighborhood leaks its stale version because the \ + fresh row is evicted from the over-fetched top-k, so WithinSourceDedupExec never \ + sees it. Un-ignore once the active arm gains a predicate-independent recency \ + filter over the whole memtable."] + async fn test_vector_search_active_stale_update_out_of_neighborhood() { + // BUG REPRODUCTION (vector case: a PK update that moves out of the neighborhood). + // + // Within a *single* active memtable, pk=1 is first inserted ON the query + // (distance ~0), then updated to a FAR vector. The append-only HNSW keeps + // both nodes live. WithinSourceDedupExec(KeepMaxRowAddr) only collapses + // duplicate PKs that are BOTH present in the over-fetched candidate set. + // + // Here the fresh (far) pk=1 is evicted from the candidate set — there are + // enough nearer filler rows that it ranks below the fetch cutoff — so the + // dedup never sees it and the STALE near pk=1 leaks as the nearest hit. + // This is the predicate-crossing hole: the row that *would* suppress the + // stale version isn't in the result set, so result-set dedup can't help. + // + // Desired (NewestPkFilterExec) behaviour: pk=1's newest row-position is + // the far one, computed predicate-independently over the whole memtable, + // so the stale near node is dropped and pk=1 must NOT surface at ~0. + use crate::dataset::mem_wal::scanner::collector::{InMemoryMemTableRef, InMemoryMemTables}; + use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; + use datafusion::prelude::SessionContext; + use futures::TryStreamExt; + + let schema = create_vector_schema(); + let temp_dir = tempfile::tempdir().unwrap(); + let base_uri = format!("{}/base", temp_dir.path().to_str().unwrap()); + + let batch_store = Arc::new(BatchStore::with_capacity(16)); + let mut index_store = IndexStore::new(); + index_store.add_hnsw( + "vector_hnsw".to_string(), + 1, + "vector".to_string(), + lance_linalg::distance::DistanceType::L2, + 64, + 8, + ); + + // First append: stale pk=1 ON the query, plus five filler rows strictly + // farther than pk=1 but far nearer than the eventual fresh pk=1. + let q = [0.1, 0.2, 0.3, 0.4]; + let stale_then_fillers = batch_rows( + &schema, + &[ + (1, q), + (10, [0.11, 0.21, 0.31, 0.41]), + (11, [0.13, 0.23, 0.33, 0.43]), + (12, [0.15, 0.25, 0.35, 0.45]), + (13, [0.17, 0.27, 0.37, 0.47]), + (14, [0.19, 0.29, 0.39, 0.49]), + ], + ); + let (_, _, bp0) = batch_store.append(stale_then_fillers.clone()).unwrap(); + index_store + .insert_with_batch_position(&stale_then_fillers, 0, Some(bp0)) + .unwrap(); + + // Second append: the UPDATE — pk=1 moved far from the query. This is the + // newest version (largest row position) but it sits well outside top-k. + let fresh_pk1 = batch_rows(&schema, &[(1, [9.0, 9.0, 9.0, 9.0])]); + let (_, _, bp1) = batch_store.append(fresh_pk1.clone()).unwrap(); + index_store + .insert_with_batch_position(&fresh_pk1, 1, Some(bp1)) + .unwrap(); + let index_store = Arc::new(index_store); + + let shard_id = uuid::Uuid::new_v4(); + let collector = LsmDataSourceCollector::without_base_table(base_uri, vec![]) + .with_in_memory_memtables( + shard_id, + InMemoryMemTables { + active: InMemoryMemTableRef { + batch_store, + index_store, + schema: schema.clone(), + generation: 1, + }, + frozen: vec![], + }, + ); + + let planner = LsmVectorSearchPlanner::new( + collector, + vec!["id".to_string()], + schema, + "vector".to_string(), + lance_linalg::distance::DistanceType::L2, + ); + + // k=3, no over-fetch: the candidate set is {pk1@near, two nearest + // fillers}; fresh pk1@far ranks 7th and never enters the candidates. + let query = create_query_vector(); + let plan = planner + .plan_search(&query, 3, 1, None, false, 1.0) + .await + .unwrap(); + let ctx = SessionContext::new(); + let stream = plan.execute(0, ctx.task_ctx()).unwrap(); + let batches: Vec = stream.try_collect().await.unwrap(); + let rows = collect_id_dist(&batches); + + assert!( + !rows.iter().any(|&(id, d)| id == 1 && d.abs() < 1e-3), + "stale near pk=1 leaked: its live vector is far from the query, so it \ + must not appear at distance ~0. results={:?}", + rows + ); + } + #[tokio::test] async fn test_vector_search_stale_read_when_fresh_falls_out_of_top_k() { // Regression for the cross-generation stale-read gap that the From 9150c072477c89b487698630fdd51c4cee7f4558 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 2 Jun 2026 20:17:12 -0500 Subject: [PATCH 02/12] fix(mem_wal): dedup predicate-crossing stale reads in active-memtable vector/FTS search MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The active memtable's HNSW (vector) and inverted (FTS) indexes are append-only, so an updated row's old index entries stay live. Both arms deduped results with WithinSourceDedupExec, which only suppresses a stale row when the fresh version is also in the result set. When an update moves a row out of the query's match set (vector: far from the query; FTS: new text no longer matches), the fresh version isn't returned, so the stale version leaked. Fix: maintain a per-memtable MVCC PK-position index — a lock-free arena skiplist keyed on (compute_pk_hash(pk_columns), row_position), enabled on the active memtable and carried through freeze. This reuses the exact recency primitive point-lookup already trusts (get_newest_visible), so the fix is "make the index arms do the seek point-lookup already does": - NewestPkFilterExec keeps an index hit iff pk_position_index.get_newest_visible(pk_hash, max_visible) == row_position, predicate-independent and snapshot-exact (it keys on the same max_visible the scanner latched, read back from the scanner, not a fresh load, to avoid a concurrent-append race). - Wired into the active vector arm (replacing WithinSourceDedupExec) and the active FTS arm (adding with_row_id + the filter). - The cross-source block-list now sources in-memory membership from the same index (in_memory_pk_hashes -> PkPositionIndex::pk_hashes), falling back to the BatchStore scan only when no PK index is present. Keyset == whole-store membership, so behavior is preserved; on-disk tiers unchanged. The hash keying covers single and composite primary keys uniformly and matches the hashing WithinSourceDedupExec/block_list already use. Both previously-ignored repro tests now pass; adds pk_position unit tests and an index-sourced block-list test. Co-Authored-By: Claude Opus 4.8 (1M context) --- rust/lance/src/dataset/mem_wal/index.rs | 46 +++ .../src/dataset/mem_wal/index/pk_position.rs | 201 +++++++++++++ .../mem_wal/memtable/scanner/builder.rs | 8 + .../src/dataset/mem_wal/scanner/block_list.rs | 91 +++++- .../lance/src/dataset/mem_wal/scanner/exec.rs | 3 + .../mem_wal/scanner/exec/newest_pk_filter.rs | 268 ++++++++++++++++++ .../src/dataset/mem_wal/scanner/fts_search.rs | 26 +- .../dataset/mem_wal/scanner/vector_search.rs | 77 +++-- rust/lance/src/dataset/mem_wal/write.rs | 32 ++- 9 files changed, 702 insertions(+), 50 deletions(-) create mode 100644 rust/lance/src/dataset/mem_wal/index/pk_position.rs create mode 100644 rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs diff --git a/rust/lance/src/dataset/mem_wal/index.rs b/rust/lance/src/dataset/mem_wal/index.rs index 116ea6c60ce..dff7525bc5d 100644 --- a/rust/lance/src/dataset/mem_wal/index.rs +++ b/rust/lance/src/dataset/mem_wal/index.rs @@ -18,6 +18,7 @@ mod arena_skiplist; mod btree; mod fts; mod hnsw; +mod pk_position; use std::collections::HashMap; use std::sync::atomic::{AtomicUsize, Ordering}; @@ -44,6 +45,7 @@ pub type RowPosition = u64; pub use btree::{BTreeIndexConfig, BTreeMemIndex}; pub use fts::{FtsIndexConfig, FtsMemIndex, FtsQueryExpr, SearchOptions}; pub use hnsw::{HnswIndexConfig, HnswMemIndex}; +pub use pk_position::PkPositionIndex; // ============================================================================ // Index Store @@ -201,6 +203,11 @@ pub struct IndexStore { hnsw_indexes: HashMap, /// FTS indexes keyed by index name. fts_indexes: HashMap, + /// MVCC primary-key → newest-position index. Present when the memtable has a + /// primary key; maintained on every insert and read by the active vector / + /// FTS arms to drop predicate-crossing stale rows. Not an `Option<&str>`-keyed + /// map like the others — there is at most one per memtable. + pk_position_index: Option, /// Maximum batch position that is durable in the WAL and therefore /// visible to scanners. Advanced unconditionally after a WAL append /// succeeds; not gated on whether any indexes are configured. @@ -213,6 +220,7 @@ impl Default for IndexStore { btree_indexes: HashMap::new(), hnsw_indexes: HashMap::new(), fts_indexes: HashMap::new(), + pk_position_index: None, max_visible_batch_position: AtomicUsize::new(0), } } @@ -230,6 +238,7 @@ impl std::fmt::Debug for IndexStore { &self.hnsw_indexes.keys().collect::>(), ) .field("fts_indexes", &self.fts_indexes.keys().collect::>()) + .field("pk_position_index", &self.pk_position_index) .field( "max_visible_batch_position", &self.max_visible_batch_position.load(Ordering::Acquire), @@ -362,6 +371,24 @@ impl IndexStore { .insert(name, FtsMemIndex::with_params(field_id, column, params)); } + /// Enable the MVCC primary-key → newest-position index over `pk_columns`. + /// + /// Build this once at construction, before any inserts (it is maintained on + /// every subsequent insert). A no-op when `pk_columns` is empty (a memtable + /// without a primary key has nothing to dedup). The active vector / FTS + /// search arms read it via [`Self::pk_position_index`] to drop stale rows. + pub fn enable_pk_position_index(&mut self, pk_columns: Vec) { + if !pk_columns.is_empty() { + self.pk_position_index = Some(PkPositionIndex::new(pk_columns)); + } + } + + /// The MVCC primary-key → newest-position index, if the memtable has a + /// primary key (see [`Self::enable_pk_position_index`]). + pub fn pk_position_index(&self) -> Option<&PkPositionIndex> { + self.pk_position_index.as_ref() + } + /// Insert a batch into all indexes. pub fn insert(&self, batch: &RecordBatch, row_offset: u64) -> Result<()> { self.insert_with_batch_position(batch, row_offset, None) @@ -384,6 +411,9 @@ impl IndexStore { for index in self.fts_indexes.values() { index.insert(batch, row_offset)?; } + if let Some(pk_index) = &self.pk_position_index { + pk_index.insert(batch, row_offset)?; + } // Update global watermark after all indexes have been updated if let Some(bp) = batch_position { @@ -440,6 +470,13 @@ impl IndexStore { } } + // PK-position index: one entry per row, in batch order. + if let Some(pk_index) = &self.pk_position_index { + for stored in batches { + pk_index.insert(&stored.data, stored.row_offset)?; + } + } + // Update global watermark to the max batch position let max_bp = batches.iter().map(|b| b.batch_position).max().unwrap(); self.advance_max_visible_batch_position(max_bp); @@ -552,6 +589,15 @@ impl IndexStore { .map(|(name, _idx_type, duration)| (name.to_string(), duration)) .collect(); + // PK-position index: cheap (one skiplist insert per row), updated + // inline after the parallel index threads join so it is in place + // before the visibility watermark advances below. + if let Some(pk_index) = &self.pk_position_index { + for stored in batches { + pk_index.insert(&stored.data, stored.row_offset)?; + } + } + // Update global watermark to the max batch position let max_bp = batches.iter().map(|b| b.batch_position).max().unwrap(); self.advance_max_visible_batch_position(max_bp); diff --git a/rust/lance/src/dataset/mem_wal/index/pk_position.rs b/rust/lance/src/dataset/mem_wal/index/pk_position.rs new file mode 100644 index 00000000000..26c17794239 --- /dev/null +++ b/rust/lance/src/dataset/mem_wal/index/pk_position.rs @@ -0,0 +1,201 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright The Lance Authors + +//! Maintained MVCC primary-key → newest-position index for in-memory memtables. +//! +//! Keyed on `(pk_hash, row_position)` in the same lock-free arena skiplist the +//! [`super::BTreeMemIndex`] uses, so every version of a primary key is a +//! distinct, position-ordered entry — an MVCC version chain +//! `(k,p0),(k,p1),(k,p2)…`. [`Self::get_newest_visible`] answers "the newest +//! row position of this PK that is visible at the watermark" with a single +//! seek-and-stop, the same primitive `BTreeMemIndex::get_newest_visible` +//! exposes and that point-lookup already trusts — but keyed on +//! [`compute_pk_hash`] rather than a single column, so it covers composite and +//! otherwise-unindexed primary keys uniformly. +//! +//! The active vector / FTS search arms use this to drop a stale hit whose PK +//! has a newer version that the (append-only) secondary index didn't return — +//! the predicate-crossing stale read those arms otherwise leak. Because the row +//! position is itself the MVCC version stamp, a reader filtering on its latched +//! `max_visible` watermark is unaffected by concurrent appends (which only add +//! larger positions), so no snapshot needs to be co-published with the query. + +use std::collections::HashSet; +use std::sync::Mutex; + +use arrow_array::RecordBatch; +use lance_core::{Error, Result}; + +use super::RowPosition; +use super::arena_skiplist::{SkipListReader, SkipListWriter, new_skiplist}; +use crate::dataset::mem_wal::scanner::exec::{compute_pk_hash, resolve_pk_indices}; + +/// Skiplist key: `(pk_hash, row_position)`. Sorting by hash then position means +/// a seek to `(hash, watermark)` lands on the newest version of that hash at or +/// below the watermark. The row position keeps every entry unique. +#[derive(Clone, Copy, PartialEq, Eq, PartialOrd, Ord)] +struct PkPosKey { + hash: u64, + position: RowPosition, +} + +/// Append-only, lock-free-read index from `compute_pk_hash(pk_columns)` to the +/// row positions that key was written at. Single-writer (the MemTable serializes +/// inserts behind an uncontended `Mutex`); reads take no lock. +pub struct PkPositionIndex { + reader: SkipListReader, + writer: Mutex>, + /// Primary-key column names, resolved to indices against each batch's schema. + pk_columns: Vec, +} + +impl std::fmt::Debug for PkPositionIndex { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("PkPositionIndex") + .field("pk_columns", &self.pk_columns) + .field("len", &self.len()) + .finish() + } +} + +impl PkPositionIndex { + /// Create an index over `pk_columns` (the unenforced primary key). + pub fn new(pk_columns: Vec) -> Self { + let (writer, reader) = new_skiplist::(); + Self { + reader, + writer: Mutex::new(writer), + pk_columns, + } + } + + /// Insert every row's `(pk_hash, row_offset + row_idx)`. + pub fn insert(&self, batch: &RecordBatch, row_offset: u64) -> Result<()> { + if batch.num_rows() == 0 { + return Ok(()); + } + let pk_indices = resolve_pk_indices(batch, &self.pk_columns) + .map_err(|e| Error::invalid_input(e.to_string()))?; + let mut writer = self.writer.lock().unwrap(); + for row in 0..batch.num_rows() { + let hash = compute_pk_hash(batch, &pk_indices, row); + writer.insert(PkPosKey { + hash, + position: row_offset + row as u64, + }); + } + Ok(()) + } + + /// The newest row position written for `pk_hash` that is `<= max_visible_row` + /// (inclusive), or `None` if the key has no visible version. A single + /// seek-and-stop on the skiplist (largest key `<= (pk_hash, max_visible_row)`) + /// — no range collect, no allocation. + pub fn get_newest_visible( + &self, + pk_hash: u64, + max_visible_row: RowPosition, + ) -> Option { + let target = PkPosKey { + hash: pk_hash, + position: max_visible_row, + }; + self.reader + .upper_bound_with(&target, |key| (key.hash == pk_hash).then_some(key.position)) + .flatten() + } + + /// Every distinct primary-key hash currently in the index. The cross-source + /// block-list uses this as an in-memory generation's membership set instead + /// of re-scanning (and re-hashing) the `BatchStore` per query. + pub fn pk_hashes(&self) -> HashSet { + self.reader.iter().map(|key| key.hash).collect() + } + + /// Number of entries (one per inserted row, not per distinct key). + pub fn len(&self) -> usize { + self.reader.len() + } + + /// Whether the index has no entries. + pub fn is_empty(&self) -> bool { + self.len() == 0 + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow_array::{Int32Array, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + use std::sync::Arc; + + fn schema() -> Arc { + Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, true), + ])) + } + + fn batch(ids: &[i32], names: &[&str]) -> RecordBatch { + RecordBatch::try_new( + schema(), + vec![ + Arc::new(Int32Array::from(ids.to_vec())), + Arc::new(StringArray::from(names.to_vec())), + ], + ) + .unwrap() + } + + /// Hash a single-column `id` PK the way the index does, so a test can probe + /// `get_newest_visible` by value. + fn hash_id(id: i32) -> u64 { + let b = batch(&[id], &["x"]); + let pk_indices = resolve_pk_indices(&b, &["id".to_string()]).unwrap(); + compute_pk_hash(&b, &pk_indices, 0) + } + + #[test] + fn newest_visible_tracks_updates_under_watermark() { + let index = PkPositionIndex::new(vec!["id".to_string()]); + // id=1 at positions 0 and 3 (an update); id=2 at position 1. + index.insert(&batch(&[1, 2], &["a", "b"]), 0).unwrap(); + index.insert(&batch(&[1], &["a2"]), 3).unwrap(); + + // Watermark above the update sees the newest position. + assert_eq!(index.get_newest_visible(hash_id(1), 5), Some(3)); + assert_eq!(index.get_newest_visible(hash_id(2), 5), Some(1)); + // Watermark below the update hides it — the older position wins. + assert_eq!(index.get_newest_visible(hash_id(1), 2), Some(0)); + // Watermark below every version of a key. + assert_eq!(index.get_newest_visible(hash_id(1), 0), Some(0)); + assert_eq!(index.get_newest_visible(hash_id(2), 0), None); + // Absent key. + assert_eq!(index.get_newest_visible(hash_id(999), 5), None); + } + + #[test] + fn composite_pk_is_hashed_as_a_tuple() { + // Two-column PK (id, name): (1,"a") and (1,"b") are distinct keys. + let index = PkPositionIndex::new(vec!["id".to_string(), "name".to_string()]); + index.insert(&batch(&[1, 1], &["a", "b"]), 0).unwrap(); + + let b = batch(&[1], &["a"]); + let pk_indices = resolve_pk_indices(&b, &["id".to_string(), "name".to_string()]).unwrap(); + let hash_1a = compute_pk_hash(&b, &pk_indices, 0); + let b2 = batch(&[1], &["b"]); + let hash_1b = compute_pk_hash(&b2, &pk_indices, 0); + + assert_eq!(index.get_newest_visible(hash_1a, 10), Some(0)); + assert_eq!(index.get_newest_visible(hash_1b, 10), Some(1)); + assert_ne!(hash_1a, hash_1b); + } + + #[test] + fn empty_batch_is_a_noop() { + let index = PkPositionIndex::new(vec!["id".to_string()]); + index.insert(&batch(&[], &[]), 0).unwrap(); + assert!(index.is_empty()); + } +} diff --git a/rust/lance/src/dataset/mem_wal/memtable/scanner/builder.rs b/rust/lance/src/dataset/mem_wal/memtable/scanner/builder.rs index 2c5192e28a1..17fa9c76a65 100644 --- a/rust/lance/src/dataset/mem_wal/memtable/scanner/builder.rs +++ b/rust/lance/src/dataset/mem_wal/memtable/scanner/builder.rs @@ -366,6 +366,14 @@ impl MemTableScanner { self } + /// The `max_visible_batch_position` snapshot this scanner latched at + /// construction. A downstream recency filter must key on this same snapshot + /// (not a fresh read of the IndexStore watermark, which a concurrent append + /// could have advanced) so it stays consistent with the rows the search saw. + pub fn max_visible_batch_position(&self) -> usize { + self.max_visible_batch_position + } + /// Include the _rowaddr column in output. /// /// Same value as _rowid but named for compatibility with LSM scanner. diff --git a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs index 684fde48da1..31e5ffe8001 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs @@ -25,7 +25,7 @@ use super::data_source::{LsmDataSource, LsmGeneration}; use super::exec::{compute_pk_hash, resolve_pk_indices}; use super::flushed_cache::{FlushedMemTableCache, open_flushed_dataset}; use crate::dataset::Dataset; -use crate::dataset::mem_wal::write::BatchStore; +use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; use crate::session::Session; /// Per-source blocked PK-hash sets, keyed by `(shard_id, generation)`. Each @@ -55,11 +55,12 @@ pub async fn compute_source_block_lists( LsmDataSource::BaseTable { .. } => has_base = true, LsmDataSource::ActiveMemTable { batch_store, + index_store, shard_id, generation, .. } => { - let hashes = Arc::new(pk_hashes_from_batch_store(batch_store, pk_columns)?); + let hashes = Arc::new(in_memory_pk_hashes(batch_store, index_store, pk_columns)?); by_shard .entry(*shard_id) .or_default() @@ -119,9 +120,11 @@ pub async fn fresh_tier_block_list( for source in sources { let set = match source { LsmDataSource::BaseTable { .. } => continue, - LsmDataSource::ActiveMemTable { batch_store, .. } => { - Arc::new(pk_hashes_from_batch_store(batch_store, pk_columns)?) - } + LsmDataSource::ActiveMemTable { + batch_store, + index_store, + .. + } => Arc::new(in_memory_pk_hashes(batch_store, index_store, pk_columns)?), LsmDataSource::FlushedMemTable { path, .. } => { flushed_pk_hashes(path, pk_columns, session, flushed_cache).await? } @@ -133,6 +136,25 @@ pub async fn fresh_tier_block_list( Ok(sets) } +/// PK-hash membership of an in-memory (active / frozen) memtable. +/// +/// Reads it straight from the memtable's maintained MVCC PK-position index — its +/// keyset *is* the membership set, already hashed, so there is nothing to +/// re-scan or re-hash. Falls back to a one-time `BatchStore` scan only when the +/// memtable has no PK-position index (e.g. a table without a primary key), which +/// the production vector-search path never hits since that index is always +/// enabled alongside the secondary indexes. +fn in_memory_pk_hashes( + batch_store: &BatchStore, + index_store: &IndexStore, + pk_columns: &[String], +) -> Result> { + match index_store.pk_position_index() { + Some(index) => Ok(index.pk_hashes()), + None => pk_hashes_from_batch_store(batch_store, pk_columns), + } +} + /// Hash the PK membership of an in-memory memtable (active or frozen) from its /// committed `BatchStore` rows. pub fn pk_hashes_from_batch_store( @@ -457,4 +479,63 @@ mod tests { assert!(!blocked.contains_key(&(Some(a), g2))); assert!(!blocked.contains_key(&(Some(b), g2))); } + + #[tokio::test] + async fn in_memory_membership_reads_from_pk_position_index() { + // When the memtable has a maintained PK-position index, the block-list + // sources its membership from that index (no BatchStore re-scan) and + // still suppresses an older generation's stale copy. + use crate::dataset::mem_wal::scanner::data_source::{LsmDataSource, LsmGeneration}; + use crate::dataset::mem_wal::write::IndexStore; + use uuid::Uuid; + + let shard = Uuid::new_v4(); + + // Frozen gen 1: stale pk=1, no PK-position index (exercises the fallback). + let stale_store = BatchStore::with_capacity(8); + stale_store.append(id_batch(&[1])).unwrap(); + + // Active gen 2: pk=1 re-written + pk=2, with the index enabled + populated. + let active_store = BatchStore::with_capacity(8); + let mut active_index = IndexStore::new(); + active_index.enable_pk_position_index(vec!["id".to_string()]); + active_index.insert(&id_batch(&[1]), 0).unwrap(); + active_index.insert(&id_batch(&[2]), 1).unwrap(); + active_store.append(id_batch(&[1])).unwrap(); + active_store.append(id_batch(&[2])).unwrap(); + + let schema = id_batch(&[1]).schema(); + let sources = vec![ + LsmDataSource::ActiveMemTable { + batch_store: Arc::new(stale_store), + index_store: Arc::new(IndexStore::new()), + schema: schema.clone(), + shard_id: shard, + generation: LsmGeneration::memtable(1), + }, + LsmDataSource::ActiveMemTable { + batch_store: Arc::new(active_store), + index_store: Arc::new(active_index), + schema, + shard_id: shard, + generation: LsmGeneration::memtable(2), + }, + ]; + + let blocked = Box::pin(compute_source_block_lists( + &sources, + &["id".to_string()], + None, + None, + )) + .await + .unwrap(); + + // gen 2's index-sourced membership is {pk=1, pk=2}; gen 1 (stale pk=1) + // is blocked on both, and the newest gen 2 has no blocked set. + let g1 = LsmGeneration::memtable(1); + assert!(blocks(&blocked[&(Some(shard), g1)], 1)); + assert!(blocks(&blocked[&(Some(shard), g1)], 2)); + assert!(!blocked.contains_key(&(Some(shard), LsmGeneration::memtable(2)))); + } } diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec.rs b/rust/lance/src/dataset/mem_wal/scanner/exec.rs index 88fd617dc0a..a51028dc0c3 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec.rs @@ -11,10 +11,12 @@ //! - [`CoalesceFirstExec`]: Returns first non-empty result with short-circuit //! - [`WithinSourceDedupExec`]: Deduplicates rows with the same PK from a single source //! - [`PkHashFilterExec`]: Drops rows whose PK hash was superseded by a newer generation (the cross-generation block-list) +//! - [`NewestPkFilterExec`]: Drops active-memtable hits that aren't the newest visible version of their PK (the within-source recency filter) mod bloom_guard; mod coalesce_first; mod generation_tag; +mod newest_pk_filter; mod pk; mod pk_hash_filter; mod within_source_dedup; @@ -22,6 +24,7 @@ mod within_source_dedup; pub use bloom_guard::{BloomFilterGuardExec, compute_pk_hash_from_scalars}; pub use coalesce_first::CoalesceFirstExec; pub use generation_tag::{MEMTABLE_GEN_COLUMN, MemtableGenTagExec}; +pub use newest_pk_filter::NewestPkFilterExec; pub use pk::{ ROW_ADDRESS_COLUMN, compute_pk_hash, is_supported_pk_type, resolve_pk_indices, validate_pk_types, diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs new file mode 100644 index 00000000000..9bf1aa9baec --- /dev/null +++ b/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs @@ -0,0 +1,268 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright The Lance Authors + +//! Drop predicate-crossing stale rows from an active-memtable index search. +//! +//! The active memtable's HNSW / inverted index are append-only, so an updated +//! row's old entries stay live. When an update moves a row out of the query's +//! match set, the fresh version isn't in the index result, so the result-set +//! dedup ([`super::WithinSourceDedupExec`]) has nothing to suppress the stale +//! version against — and it leaks. +//! +//! This node closes that hole with a predicate-independent recency check: for +//! each hit it asks the memtable's maintained MVCC PK-position index +//! ([`crate::dataset::mem_wal::index::PkPositionIndex`]) for the newest position +//! of that hit's primary key visible at the query's `max_visible` watermark, and +//! keeps the hit **iff that equals the hit's own row position**. A stale hit +//! (some newer version exists) is dropped even when that newer version never +//! appears in the result. This is exactly the seek point-lookup already does; +//! the index search arms simply didn't do it. + +use std::any::Any; +use std::fmt; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use arrow::compute::filter_record_batch; +use arrow_array::{Array, BooleanArray, RecordBatch, UInt64Array}; +use arrow_schema::SchemaRef; +use datafusion::error::{DataFusionError, Result as DFResult}; +use datafusion::execution::TaskContext; +use datafusion::physical_expr::EquivalenceProperties; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties, + SendableRecordBatchStream, +}; +use futures::{Stream, StreamExt}; + +use super::pk::{compute_pk_hash, resolve_pk_indices}; +use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; + +/// Keeps only the index hits that are the newest visible version of their PK. +/// +/// The input must expose all `pk_columns` and the `row_id_column` (`UInt64`, +/// the BatchStore row position). The output schema is unchanged. +pub struct NewestPkFilterExec { + input: Arc, + pk_columns: Vec, + row_id_column: String, + /// Holds the maintained `PkPositionIndex` queried per hit. + index_store: Arc, + /// Resolves the `max_visible` row watermark from the visible batch prefix. + batch_store: Arc, + /// The MVCC batch-position snapshot the index search latched. Captured once + /// at plan time and shared with the search so the recency check keys on the + /// same snapshot the hits came from. + max_visible_batch_position: usize, + properties: Arc, +} + +impl fmt::Debug for NewestPkFilterExec { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + // `BatchStore` / `IndexStore` aren't `Debug`; show only the knobs. + f.debug_struct("NewestPkFilterExec") + .field("pk_columns", &self.pk_columns) + .field("row_id_column", &self.row_id_column) + .field( + "max_visible_batch_position", + &self.max_visible_batch_position, + ) + .finish() + } +} + +impl NewestPkFilterExec { + pub fn new( + input: Arc, + pk_columns: Vec, + row_id_column: impl Into, + index_store: Arc, + batch_store: Arc, + max_visible_batch_position: usize, + ) -> Self { + // A filter preserves the input schema and partitioning. + let properties = Arc::new(PlanProperties::new( + EquivalenceProperties::new(input.schema()), + input.output_partitioning().clone(), + input.pipeline_behavior(), + input.boundedness(), + )); + Self { + input, + pk_columns, + row_id_column: row_id_column.into(), + index_store, + batch_store, + max_visible_batch_position, + properties, + } + } + + /// The inclusive max visible row position for this snapshot, or `None` when + /// no rows are visible. Mirrors `point_lookup::probe_position`: the visible + /// prefix is `[0, max_visible_batch_position]`, and each batch carries its + /// cumulative `row_offset`. + fn max_visible_row(&self) -> Option { + let len = self.batch_store.len(); + if len == 0 { + return None; + } + let last_visible_idx = self.max_visible_batch_position.min(len - 1); + let last = self.batch_store.get(last_visible_idx)?; + let visible_end = last.row_offset + last.num_rows as u64; // exclusive + visible_end.checked_sub(1) + } +} + +impl DisplayAs for NewestPkFilterExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default + | DisplayFormatType::Verbose + | DisplayFormatType::TreeRender => { + write!( + f, + "NewestPkFilterExec: pk=[{}], row_id={}, max_visible_batch={}", + self.pk_columns.join(", "), + self.row_id_column, + self.max_visible_batch_position, + ) + } + } + } +} + +impl ExecutionPlan for NewestPkFilterExec { + fn name(&self) -> &str { + "NewestPkFilterExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.input.schema() + } + + fn properties(&self) -> &Arc { + &self.properties + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> DFResult> { + if children.len() != 1 { + return Err(DataFusionError::Internal( + "NewestPkFilterExec requires exactly one child".to_string(), + )); + } + Ok(Arc::new(Self::new( + children[0].clone(), + self.pk_columns.clone(), + self.row_id_column.clone(), + self.index_store.clone(), + self.batch_store.clone(), + self.max_visible_batch_position, + ))) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> DFResult { + let input_stream = self.input.execute(partition, context)?; + Ok(Box::pin(NewestPkFilterStream { + input: input_stream, + pk_columns: self.pk_columns.clone(), + row_id_column: self.row_id_column.clone(), + index_store: self.index_store.clone(), + max_visible_row: self.max_visible_row(), + schema: self.schema(), + })) + } +} + +struct NewestPkFilterStream { + input: SendableRecordBatchStream, + pk_columns: Vec, + row_id_column: String, + index_store: Arc, + /// Inclusive watermark snapshot; `None` when no rows are visible. + max_visible_row: Option, + schema: SchemaRef, +} + +impl NewestPkFilterStream { + fn filter_batch(&self, batch: RecordBatch) -> DFResult { + // No PK-position index (memtable without a primary key), no visible + // rows, or an empty batch: nothing to dedup against, so pass it through. + let Some(pk_index) = self.index_store.pk_position_index() else { + return Ok(batch); + }; + let Some(max_visible_row) = self.max_visible_row else { + return Ok(batch); + }; + if batch.num_rows() == 0 { + return Ok(batch); + } + + let pk_indices = resolve_pk_indices(&batch, &self.pk_columns)?; + let row_ids = batch + .column_by_name(&self.row_id_column) + .ok_or_else(|| { + DataFusionError::Internal(format!( + "Row-id column '{}' not found in NewestPkFilterExec input", + self.row_id_column + )) + })? + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Internal(format!( + "Row-id column '{}' is not UInt64", + self.row_id_column + )) + })?; + + let keep: BooleanArray = (0..batch.num_rows()) + .map(|row| { + // A null row position can't be ordered; keep it rather than + // guess (callers always project a real position here). + if row_ids.is_null(row) { + return true; + } + let position = row_ids.value(row); + let hash = compute_pk_hash(&batch, &pk_indices, row); + // Keep iff this hit is the newest visible version of its PK. + pk_index.get_newest_visible(hash, max_visible_row) == Some(position) + }) + .collect(); + filter_record_batch(&batch, &keep) + .map_err(|e| DataFusionError::ArrowError(Box::new(e), None)) + } +} + +impl Stream for NewestPkFilterStream { + type Item = DFResult; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + match self.input.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => Poll::Ready(Some(self.filter_batch(batch))), + other => other, + } + } +} + +impl datafusion::physical_plan::RecordBatchStream for NewestPkFilterStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} diff --git a/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs b/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs index 7ff800edcc8..490a34ee366 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs @@ -51,6 +51,7 @@ use tracing::instrument; use super::collector::LsmDataSourceCollector; use super::data_source::LsmDataSource; +use super::exec::NewestPkFilterExec; use super::flushed_cache::{FlushedMemTableCache, open_flushed_dataset}; use super::projection::project_to_canonical; use crate::dataset::mem_wal::memtable::scanner::MemTableScanner; @@ -232,6 +233,12 @@ impl LsmFtsSearchPlanner { MemTableScanner::new(batch_store.clone(), index_store.clone(), schema.clone()); let cols = self.fts_scanner_projection(projection); scanner.project(&cols.iter().map(|s| s.as_str()).collect::>()); + // Expose the row position so the recency filter can identify the + // newest visible version of each PK. The append-only inverted + // index keeps an updated row's old postings live, so a stale hit + // can match a query the fresh row no longer does; the filter + // drops it. `project_to_canonical` strips `_rowid` afterward. + scanner.with_row_id(); // `MemTableScanner::full_text_search` takes a raw match // string; richer query shapes (phrase/boolean/fuzzy) can // be plumbed through once the MemTable scanner accepts a @@ -250,7 +257,16 @@ impl LsmFtsSearchPlanner { // today; the per-partition Sort+fetch above bounds the // emitted rows. let _ = k; - scanner.create_plan().await + let plan = scanner.create_plan().await?; + let filtered: Arc = Arc::new(NewestPkFilterExec::new( + plan, + self.pk_columns.clone(), + lance_core::ROW_ID, + index_store.clone(), + batch_store.clone(), + scanner.max_visible_batch_position(), + )); + Ok(filtered) } } } @@ -404,6 +420,7 @@ mod tests { // Active memtable with its own FTS index, containing a matching row. let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut indexes = IndexStore::new(); + indexes.enable_pk_position_index(vec!["id".to_string()]); indexes.add_fts("text_fts".to_string(), 1, "text".to_string()); let active_batch = make_batch( &schema, @@ -483,6 +500,7 @@ mod tests { let schema = fts_schema(); let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut indexes = IndexStore::new(); + indexes.enable_pk_position_index(vec!["id".to_string()]); // text column has field_id 1 in fts_schema() indexes.add_fts("text_fts".to_string(), 1, "text".to_string()); let batch = make_batch( @@ -563,11 +581,6 @@ mod tests { } #[tokio::test] - #[ignore = "known bug: active-memtable predicate-crossing stale read in the FTS arm. \ - A PK updated so its new text no longer matches the query still leaks its stale \ - version, because the append-only inverted index keeps the old posting and the \ - fresh row isn't a candidate to dedup against. Un-ignore once the active arm gains \ - a predicate-independent recency filter over the whole memtable."] async fn active_stale_update_predicate_crossing_leaks() { // BUG REPRODUCTION (FTS case: a PK update that crosses out of the match set). // @@ -584,6 +597,7 @@ mod tests { let schema = fts_schema(); let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut indexes = IndexStore::new(); + indexes.enable_pk_position_index(vec!["id".to_string()]); indexes.add_fts("text_fts".to_string(), 1, "text".to_string()); // Insert pk=1 ("alpha lance") and an unrelated live pk=2 ("alpha foo"). diff --git a/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs b/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs index a6a6ddb3538..f53d92a3dde 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs @@ -27,7 +27,6 @@ use crate::io::exec::TakeExec; use super::collector::LsmDataSourceCollector; use super::data_source::LsmDataSource; -use super::exec::{DedupDirection, WithinSourceDedupExec}; use super::flushed_cache::{FlushedMemTableCache, open_flushed_dataset}; use super::projection::{ DISTANCE_COLUMN, build_scanner_projection, canonical_output_schema, null_columns, @@ -248,7 +247,7 @@ impl LsmVectorSearchPlanner { } else { k }; - let knn = Box::pin(self.build_knn_plan( + let (knn, active_max_visible) = Box::pin(self.build_knn_plan( source, query_vector, fetch_k, @@ -259,20 +258,32 @@ impl LsmVectorSearchPlanner { .await?; // Make each source independently newest-per-PK before the union: // * active: the append-only HNSW returns one node per inserted - // version, so collapse duplicate PKs to the newest insert - // (KeepMaxRowAddr on `_rowid`) and re-sort by distance. This - // stays probabilistic — a fresh version evicted from the - // over-fetched top-k still leaks. + // version *and* leaves stale versions of updated PKs live. The + // recency filter keeps only the hit that is the newest visible + // version of its PK (per the maintained MVCC PK-position index), + // closing the predicate-crossing stale read, then re-sort by + // distance. // * flushed/base: drop cross-gen superseded rows via the // block-list (within-gen is handled by the flushed DV). let knn = if is_active { - let deduped: Arc = Arc::new(WithinSourceDedupExec::new( - knn, - self.pk_columns.clone(), - lance_core::ROW_ID, - DedupDirection::KeepMaxRowAddr, - )); - sort_by_distance(deduped, k)? + let (batch_store, index_store) = match source { + LsmDataSource::ActiveMemTable { + batch_store, + index_store, + .. + } => (batch_store.clone(), index_store.clone()), + _ => unreachable!("is_active implies ActiveMemTable"), + }; + let filtered: Arc = + Arc::new(super::exec::NewestPkFilterExec::new( + knn, + self.pk_columns.clone(), + lance_core::ROW_ID, + index_store, + batch_store, + active_max_visible.expect("active arm returns its max_visible snapshot"), + )); + sort_by_distance(filtered, k)? } else { match blocked { Some(set) => Arc::new(super::exec::PkHashFilterExec::new( @@ -369,6 +380,10 @@ impl LsmVectorSearchPlanner { } /// Build KNN plan for a single data source. + /// + /// Returns the plan and, for the active memtable, the `max_visible_batch_position` + /// snapshot its scanner latched — threaded into the recency filter so it keys + /// on the same snapshot the search saw (`None` for base / flushed sources). async fn build_knn_plan( &self, source: &LsmDataSource, @@ -377,7 +392,7 @@ impl LsmVectorSearchPlanner { nprobes: usize, projection: Option<&[String]>, refine: bool, - ) -> Result> { + ) -> Result<(Arc, Option)> { match source { LsmDataSource::BaseTable { dataset } => { let mut scanner = dataset.scan(); @@ -402,7 +417,7 @@ impl LsmVectorSearchPlanner { if refine { scanner.refine(1); } - scanner.create_plan().await + Ok((scanner.create_plan().await?, None)) } LsmDataSource::FlushedMemTable { path, .. } => { let dataset = @@ -418,7 +433,7 @@ impl LsmVectorSearchPlanner { scanner.nprobes(nprobes); scanner.distance_metric(self.distance_type); scanner.fast_search(); - scanner.create_plan().await + Ok((scanner.create_plan().await?, None)) } LsmDataSource::ActiveMemTable { batch_store, @@ -447,7 +462,9 @@ impl LsmVectorSearchPlanner { scanner.nearest(&self.vector_column, query_arr, k); scanner.nprobes(nprobes); scanner.distance_metric(self.distance_type); - scanner.create_plan().await + let plan = scanner.create_plan().await?; + // Capture the scanner's own latched snapshot for the recency filter. + Ok((plan, Some(scanner.max_visible_batch_position()))) } } } @@ -641,6 +658,7 @@ mod tests { // Active memtable with HNSW index over the "vector" column. let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -759,6 +777,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -838,6 +857,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -951,6 +971,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1007,8 +1028,7 @@ mod tests { plan_str ); assert!( - plan_str.contains("WithinSourceDedupExec") - && plan_str.contains("SortPreservingMergeExec"), + plan_str.contains("NewestPkFilterExec") && plan_str.contains("SortPreservingMergeExec"), "expected per-arm dedup + distance merge, got:\n{}", plan_str ); @@ -1091,6 +1111,7 @@ mod tests { // "right" vector close to the query, plus an unrelated pk=2. let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1210,6 +1231,7 @@ mod tests { // Active memtable: id=3 with HNSW index. let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1450,6 +1472,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1513,14 +1536,14 @@ mod tests { .await .unwrap(); - // The active arm collapses duplicate-PK HNSW nodes itself via - // WithinSourceDedupExec — there is no cross-source dedup fallback. + // The active arm collapses duplicate-PK HNSW nodes itself via the + // recency filter — there is no cross-source dedup fallback. let plan_str = format!( "{}", datafusion::physical_plan::displayable(plan.as_ref()).indent(true) ); assert!( - plan_str.contains("WithinSourceDedupExec"), + plan_str.contains("NewestPkFilterExec"), "active vector arm must self-dedup, got:\n{}", plan_str ); @@ -1550,11 +1573,6 @@ mod tests { } #[tokio::test] - #[ignore = "known bug: active-memtable predicate-crossing stale read in the vector arm. \ - A PK updated out of the query's neighborhood leaks its stale version because the \ - fresh row is evicted from the over-fetched top-k, so WithinSourceDedupExec never \ - sees it. Un-ignore once the active arm gains a predicate-independent recency \ - filter over the whole memtable."] async fn test_vector_search_active_stale_update_out_of_neighborhood() { // BUG REPRODUCTION (vector case: a PK update that moves out of the neighborhood). // @@ -1583,6 +1601,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1701,6 +1720,7 @@ mod tests { // active arm surfaces pk=2 and drops fresh pk=1. let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1897,6 +1917,7 @@ mod tests { // Active (gen 1): pk 1,2,3 re-inserted with a far vector (the fresh value). let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -2101,6 +2122,7 @@ mod tests { // Active: (1,1) re-inserted far (fresh) + an unrelated nearby (2,2). let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id1".to_string(), "id2".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -2205,6 +2227,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); + index_store.enable_pk_position_index(vec!["id".to_string()]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, diff --git a/rust/lance/src/dataset/mem_wal/write.rs b/rust/lance/src/dataset/mem_wal/write.rs index 3d59735f7e8..d2d6eb27ae8 100644 --- a/rust/lance/src/dataset/mem_wal/write.rs +++ b/rust/lance/src/dataset/mem_wal/write.rs @@ -823,6 +823,9 @@ struct SharedWriterState { config: ShardWriterConfig, schema: Arc, pk_field_ids: Vec, + /// Primary-key column names, used to (re)enable the PK-position index on + /// each fresh active memtable created at freeze. + pk_columns: Vec, max_memtable_batches: usize, max_memtable_rows: usize, index_configs: Vec, @@ -838,6 +841,7 @@ impl SharedWriterState { config: ShardWriterConfig, schema: Arc, pk_field_ids: Vec, + pk_columns: Vec, max_memtable_batches: usize, max_memtable_rows: usize, index_configs: Vec, @@ -850,6 +854,7 @@ impl SharedWriterState { config, schema, pk_field_ids, + pk_columns, max_memtable_batches, max_memtable_rows, index_configs, @@ -876,12 +881,13 @@ impl SharedWriterState { )?; if !self.index_configs.is_empty() { - let indexes = Arc::new(IndexStore::from_configs( + let mut indexes = IndexStore::from_configs( &self.index_configs, self.max_memtable_rows, self.max_memtable_batches, - )?); - new_memtable.set_indexes_arc(indexes); + )?; + indexes.enable_pk_position_index(self.pk_columns.clone()); + new_memtable.set_indexes_arc(Arc::new(indexes)); } let mut old_memtable = std::mem::replace(&mut state.memtable, new_memtable); @@ -1249,11 +1255,9 @@ impl ShardWriter { ) -> Result { // Create MemTable with primary key field IDs from schema let lance_schema = Schema::try_from(schema.as_ref())?; - let pk_field_ids: Vec = lance_schema - .unenforced_primary_key() - .iter() - .map(|f| f.id) - .collect(); + let pk_fields = lance_schema.unenforced_primary_key(); + let pk_field_ids: Vec = pk_fields.iter().map(|f| f.id).collect(); + let pk_columns: Vec = pk_fields.iter().map(|f| f.name.clone()).collect(); let mut memtable = MemTable::with_capacity( schema.clone(), manifest.current_generation, @@ -1262,14 +1266,17 @@ impl ShardWriter { config.max_memtable_batches, )?; - // Create indexes if configured and set them on the MemTable. + // Create indexes if configured and set them on the MemTable. The + // PK-position index is enabled before any WAL replay below so replayed + // rows are recorded in it. if !index_configs.is_empty() { - let indexes = Arc::new(IndexStore::from_configs( + let mut indexes = IndexStore::from_configs( index_configs, config.max_memtable_rows, config.max_memtable_batches, - )?); - memtable.set_indexes_arc(indexes); + )?; + indexes.enable_pk_position_index(pk_columns.clone()); + memtable.set_indexes_arc(Arc::new(indexes)); } // Replay any WAL entries written after the last successfully-flushed @@ -1357,6 +1364,7 @@ impl ShardWriter { config.clone(), schema.clone(), pk_field_ids, + pk_columns, config.max_memtable_batches, config.max_memtable_rows, index_configs.to_vec(), From b065eb67091d8b4ac1377bebb5d5061120108174 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 2 Jun 2026 21:02:21 -0500 Subject: [PATCH 03/12] refactor(mem_wal): route point-lookup + block-list through the PK-position index; drop WithinSourceDedupExec MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Unifies the in-memory newest-per-PK / membership decisions onto the maintained MVCC PK-position index and removes the now-redundant WithinSourceDedupExec. - point_lookup: `probe_position` prefers the value-keyed PK BTree (collision- free) and falls back to the PK-position index — hashing the key and verifying the candidate's actual value (via `visible_positions`, newest-first) to guard hash collisions — when no scalar BTree exists, so composite / unindexed PKs get the plan-free fast probe too. The plan-path active arm's WithinSourceDedupExec is replaced by `SortExec(_rowid DESC).fetch(1)` (value-exact via the filter, newest by row id). - block-list: `compute_source_block_lists` yields `Vec` instead of `Vec>`. In-memory generations are probed per candidate (`GenMembership::Index` -> `PkPositionIndex::contains_visible`, snapshot-bounded) with no per-query set materialized; flushed/base keep their cached sets, and an in-memory memtable without a PK-position index falls back to the BatchStore scan. Snapshot-bounding also closes a latent over-block where a not-yet-visible newer write could shadow an older visible copy. `PkHashFilterExec` now consumes `Vec`. - cleanup: WithinSourceDedupExec and DedupDirection (incl. the dead KeepMinRowAddr) have no remaining users and are deleted. Added shared helpers `BatchStore::max_visible_row` and point_lookup's `resolve_position`. Full mem_wal suite green; adds point-lookup-without-btree and index-sourced block-list tests. fresh_tier_block_list (external contains_pks API) still materializes sets; migrating MemTableDedupScanExec's reverse-walk HashSet onto the same probe is a benchmark-gated follow-up. Co-Authored-By: Claude Opus 4.8 (1M context) --- .../src/dataset/mem_wal/index/pk_position.rs | 35 ++ .../dataset/mem_wal/memtable/batch_store.rs | 16 + .../src/dataset/mem_wal/scanner/block_list.rs | 139 +++++- .../lance/src/dataset/mem_wal/scanner/exec.rs | 3 - .../mem_wal/scanner/exec/newest_pk_filter.rs | 20 +- .../src/dataset/mem_wal/scanner/exec/pk.rs | 2 +- .../mem_wal/scanner/exec/pk_hash_filter.rs | 50 +- .../scanner/exec/within_source_dedup.rs | 432 ------------------ .../dataset/mem_wal/scanner/point_lookup.rs | 191 ++++++-- .../dataset/mem_wal/scanner/vector_search.rs | 26 +- 10 files changed, 374 insertions(+), 540 deletions(-) delete mode 100644 rust/lance/src/dataset/mem_wal/scanner/exec/within_source_dedup.rs diff --git a/rust/lance/src/dataset/mem_wal/index/pk_position.rs b/rust/lance/src/dataset/mem_wal/index/pk_position.rs index 26c17794239..eff1e7638d3 100644 --- a/rust/lance/src/dataset/mem_wal/index/pk_position.rs +++ b/rust/lance/src/dataset/mem_wal/index/pk_position.rs @@ -105,6 +105,41 @@ impl PkPositionIndex { .flatten() } + /// Whether `pk_hash` has any version visible at `max_visible_row`. The + /// cross-source block-list's existence query — "does a newer generation + /// contain this PK?" — reduces to this, position-bounded so a not-yet-visible + /// write can't shadow an older visible copy. + pub fn contains_visible(&self, pk_hash: u64, max_visible_row: RowPosition) -> bool { + self.get_newest_visible(pk_hash, max_visible_row).is_some() + } + + /// All row positions written for `pk_hash` that are `<= max_visible_row`, in + /// ascending (oldest-first) order. Used by point-lookup to resolve a hash + /// collision: walk the matches newest-first and keep the first whose actual + /// primary-key value equals the query, so a colliding key never returns the + /// wrong row. Empty (and allocation-free past the seek) when the hash is + /// absent; the common no-collision lookup uses [`Self::get_newest_visible`]. + pub fn visible_positions( + &self, + pk_hash: u64, + max_visible_row: RowPosition, + ) -> Vec { + let start = PkPosKey { + hash: pk_hash, + position: 0, + }; + let mut positions = Vec::new(); + for key in self.reader.range_from(&start) { + if key.hash != pk_hash { + break; + } + if key.position <= max_visible_row { + positions.push(key.position); + } + } + positions + } + /// Every distinct primary-key hash currently in the index. The cross-source /// block-list uses this as an in-memory generation's membership set instead /// of re-scanning (and re-hashing) the `BatchStore` per query. diff --git a/rust/lance/src/dataset/mem_wal/memtable/batch_store.rs b/rust/lance/src/dataset/mem_wal/memtable/batch_store.rs index f4d4d797acc..3df6215d7ce 100644 --- a/rust/lance/src/dataset/mem_wal/memtable/batch_store.rs +++ b/rust/lance/src/dataset/mem_wal/memtable/batch_store.rs @@ -615,6 +615,22 @@ impl BatchStore { (0..end).collect() } + /// The inclusive maximum visible *row* position at `max_visible_batch_position`, + /// or `None` when no rows are visible. The visible batches are the committed + /// prefix `[0, last_visible_idx]`; each batch carries its cumulative + /// `row_offset`, so this is the end of the last visible batch minus one. + /// Used to bound MVCC seeks against the maintained PK-position index. + pub fn max_visible_row(&self, max_visible_batch_position: usize) -> Option { + let len = self.committed_len.load(Ordering::Acquire); + if len == 0 { + return None; + } + let last_visible_idx = max_visible_batch_position.min(len - 1); + let last = self.get(last_visible_idx)?; + let visible_end = last.row_offset + last.num_rows as u64; // exclusive + visible_end.checked_sub(1) + } + /// Check if a specific batch is visible at a given visibility position. #[inline] pub fn is_batch_visible( diff --git a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs index 31e5ffe8001..b9edcc00d0d 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs @@ -28,12 +28,69 @@ use crate::dataset::Dataset; use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; use crate::session::Session; -/// Per-source blocked PK-hash sets, keyed by `(shard_id, generation)`. Each -/// value is the membership sets of the generations newer than that source. -pub type SourceBlockLists = HashMap<(Option, LsmGeneration), Vec>>>; +/// One newer generation's PK membership, used to decide whether it shadows an +/// older source's row. In-memory generations (active / frozen) are **probed** +/// against their maintained MVCC PK-position index — no per-query set is built — +/// while on-disk generations (flushed, base) carry a materialized PK-hash set +/// (cached by path). The probe is snapshot-bounded, so a not-yet-visible write +/// can't shadow an older visible copy. +#[derive(Debug, Clone)] +pub enum GenMembership { + /// Probe the in-memory memtable's index, bounded to its visible prefix. + Index { + index_store: Arc, + /// Inclusive visible row watermark; `None` when no rows are visible. + max_visible_row: Option, + }, + /// A materialized PK-hash set (flushed / base, or an in-memory memtable that + /// has no PK-position index). + Set(Arc>), +} -/// A shard's generations paired with their PK-hash membership, before sorting. -type ShardGenSets = HashMap>)>>; +impl GenMembership { + /// Whether this generation contains `pk_hash` (visibly, for the index case). + pub fn contains(&self, pk_hash: u64) -> bool { + match self { + Self::Index { + index_store, + max_visible_row, + } => { + let Some(max) = max_visible_row else { + return false; + }; + index_store + .pk_position_index() + .is_some_and(|idx| idx.contains_visible(pk_hash, *max)) + } + Self::Set(set) => set.contains(&pk_hash), + } + } + + /// Whether this generation has no (visible) membership — used to skip adding + /// an empty blocked set. Approximate for the index case (it ignores the + /// watermark when counting), which only ever leaves a harmless no-op entry. + fn is_empty(&self) -> bool { + match self { + Self::Index { + index_store, + max_visible_row, + } => { + max_visible_row.is_none() + || index_store + .pk_position_index() + .is_none_or(|idx| idx.is_empty()) + } + Self::Set(set) => set.is_empty(), + } + } +} + +/// Per-source blocked memberships, keyed by `(shard_id, generation)`. Each value +/// is the memberships of the generations newer than that source. +pub type SourceBlockLists = HashMap<(Option, LsmGeneration), Vec>; + +/// A shard's generations paired with their membership, before sorting. +type ShardGenSets = HashMap>; /// Per-source `NEWER(G)`, keyed by `(shard_id, generation)`. Generations are /// per-shard, so a source is superseded only by strictly-newer generations of @@ -60,11 +117,11 @@ pub async fn compute_source_block_lists( generation, .. } => { - let hashes = Arc::new(in_memory_pk_hashes(batch_store, index_store, pk_columns)?); + let membership = in_memory_membership(batch_store, index_store, pk_columns)?; by_shard .entry(*shard_id) .or_default() - .push((*generation, hashes)); + .push((*generation, membership)); } LsmDataSource::FlushedMemTable { path, @@ -77,25 +134,25 @@ pub async fn compute_source_block_lists( by_shard .entry(*shard_id) .or_default() - .push((*generation, hashes)); + .push((*generation, GenMembership::Set(hashes))); } } } let mut blocked: SourceBlockLists = HashMap::new(); // Base (shardless, oldest) is superseded by every non-base generation. - let mut base_blocked: Vec>> = Vec::new(); + let mut base_blocked: Vec = Vec::new(); for (shard, mut gens) in by_shard { // Newest-first: a gen's blocked list is its own shard's newer gens. gens.sort_by_key(|(generation, _)| std::cmp::Reverse(*generation)); - let mut newer: Vec>> = Vec::new(); - for (generation, hashes) in gens { + let mut newer: Vec = Vec::new(); + for (generation, membership) in gens { if !newer.is_empty() { blocked.insert((Some(shard), generation), newer.clone()); } - if !hashes.is_empty() { - base_blocked.push(hashes.clone()); - newer.push(hashes); + if !membership.is_empty() { + base_blocked.push(membership.clone()); + newer.push(membership); } } } @@ -155,6 +212,32 @@ fn in_memory_pk_hashes( } } +/// Cross-source membership of an in-memory (active / frozen) memtable. +/// +/// Prefers a snapshot-bounded **probe** of the maintained PK-position index (no +/// per-query set built), falling back to a one-time `BatchStore` scan only when +/// the memtable has no such index (e.g. a table without a primary key) — which +/// the production vector-search path never hits, since that index is always +/// enabled alongside the secondary indexes. +fn in_memory_membership( + batch_store: &Arc, + index_store: &Arc, + pk_columns: &[String], +) -> Result { + if index_store.pk_position_index().is_some() { + let max_visible_row = batch_store.max_visible_row(index_store.max_visible_batch_position()); + Ok(GenMembership::Index { + index_store: index_store.clone(), + max_visible_row, + }) + } else { + Ok(GenMembership::Set(Arc::new(pk_hashes_from_batch_store( + batch_store, + pk_columns, + )?))) + } +} + /// Hash the PK membership of an in-memory memtable (active or frozen) from its /// committed `BatchStore` rows. pub fn pk_hashes_from_batch_store( @@ -266,9 +349,10 @@ mod tests { compute_pk_hash(&batch, &pk_indices, 0) } - /// Whether `id`'s PK hash is blocked by any of a source's newer-gen sets. - fn blocks(sets: &[Arc>], id: i32) -> bool { - sets.iter().any(|s| s.contains(&hash_id(id))) + /// Whether `id`'s PK hash is blocked by any of a source's newer-gen + /// memberships. + fn blocks(memberships: &[GenMembership], id: i32) -> bool { + memberships.iter().any(|m| m.contains(hash_id(id))) } #[test] @@ -324,10 +408,11 @@ mod tests { // One set per generation; together they cover pk=1,2,3 (not 4). assert_eq!(sets.len(), 2); + let set_blocks = |id: i32| sets.iter().any(|s| s.contains(&hash_id(id))); for id in [1, 2, 3] { - assert!(blocks(&sets, id)); + assert!(set_blocks(id)); } - assert!(!blocks(&sets, 4)); + assert!(!set_blocks(4)); } #[tokio::test] @@ -496,13 +581,21 @@ mod tests { stale_store.append(id_batch(&[1])).unwrap(); // Active gen 2: pk=1 re-written + pk=2, with the index enabled + populated. + // `insert_with_batch_position(Some(bp))` advances the visibility watermark + // so the snapshot-bounded probe sees both rows. let active_store = BatchStore::with_capacity(8); let mut active_index = IndexStore::new(); active_index.enable_pk_position_index(vec!["id".to_string()]); - active_index.insert(&id_batch(&[1]), 0).unwrap(); - active_index.insert(&id_batch(&[2]), 1).unwrap(); - active_store.append(id_batch(&[1])).unwrap(); - active_store.append(id_batch(&[2])).unwrap(); + let b1 = id_batch(&[1]); + let (bp1, off1, _) = active_store.append(b1.clone()).unwrap(); + active_index + .insert_with_batch_position(&b1, off1, Some(bp1)) + .unwrap(); + let b2 = id_batch(&[2]); + let (bp2, off2, _) = active_store.append(b2.clone()).unwrap(); + active_index + .insert_with_batch_position(&b2, off2, Some(bp2)) + .unwrap(); let schema = id_batch(&[1]).schema(); let sources = vec![ diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec.rs b/rust/lance/src/dataset/mem_wal/scanner/exec.rs index a51028dc0c3..4603f96b532 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec.rs @@ -9,7 +9,6 @@ //! - [`MemtableGenTagExec`]: Wraps a scan to add `_memtable_gen` column //! - [`BloomFilterGuardExec`]: Guards child execution with bloom filter check //! - [`CoalesceFirstExec`]: Returns first non-empty result with short-circuit -//! - [`WithinSourceDedupExec`]: Deduplicates rows with the same PK from a single source //! - [`PkHashFilterExec`]: Drops rows whose PK hash was superseded by a newer generation (the cross-generation block-list) //! - [`NewestPkFilterExec`]: Drops active-memtable hits that aren't the newest visible version of their PK (the within-source recency filter) @@ -19,7 +18,6 @@ mod generation_tag; mod newest_pk_filter; mod pk; mod pk_hash_filter; -mod within_source_dedup; pub use bloom_guard::{BloomFilterGuardExec, compute_pk_hash_from_scalars}; pub use coalesce_first::CoalesceFirstExec; @@ -30,4 +28,3 @@ pub use pk::{ validate_pk_types, }; pub use pk_hash_filter::PkHashFilterExec; -pub use within_source_dedup::{DedupDirection, WithinSourceDedupExec}; diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs index 9bf1aa9baec..6ca39c56413 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs @@ -5,9 +5,9 @@ //! //! The active memtable's HNSW / inverted index are append-only, so an updated //! row's old entries stay live. When an update moves a row out of the query's -//! match set, the fresh version isn't in the index result, so the result-set -//! dedup ([`super::WithinSourceDedupExec`]) has nothing to suppress the stale -//! version against — and it leaks. +//! match set, the fresh version isn't in the index result, so a result-set +//! dedup (keep-newest among the returned rows) has nothing to suppress the +//! stale version against — and it leaks. //! //! This node closes that hole with a predicate-independent recency check: for //! each hit it asks the memtable's maintained MVCC PK-position index @@ -100,18 +100,10 @@ impl NewestPkFilterExec { } /// The inclusive max visible row position for this snapshot, or `None` when - /// no rows are visible. Mirrors `point_lookup::probe_position`: the visible - /// prefix is `[0, max_visible_batch_position]`, and each batch carries its - /// cumulative `row_offset`. + /// no rows are visible. fn max_visible_row(&self) -> Option { - let len = self.batch_store.len(); - if len == 0 { - return None; - } - let last_visible_idx = self.max_visible_batch_position.min(len - 1); - let last = self.batch_store.get(last_visible_idx)?; - let visible_end = last.row_offset + last.num_rows as u64; // exclusive - visible_end.checked_sub(1) + self.batch_store + .max_visible_row(self.max_visible_batch_position) } } diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/pk.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/pk.rs index 523dd30bf82..94e8f0dabc4 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec/pk.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec/pk.rs @@ -4,7 +4,7 @@ //! Shared primary-key helpers for the LSM scanner execution nodes. //! //! Centralizes PK column resolution and per-row hashing so that every -//! consumer (e.g. [`super::WithinSourceDedupExec`], [`super::PkHashFilterExec`]) +//! consumer (e.g. [`super::PkHashFilterExec`], [`super::NewestPkFilterExec`]) //! resolves and hashes a primary key the same way. The row hash is kept //! consistent with the variants supported by [`super::compute_pk_hash_from_scalars`] //! so a single PK produces the same hash regardless of which exec consumes it. diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs index ee473047d01..6366b654407 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs @@ -14,7 +14,6 @@ //! when a source had >= k candidates but < k survived (over-fetch too small). use std::any::Any; -use std::collections::HashSet; use std::fmt; use std::pin::Pin; use std::sync::Arc; @@ -33,15 +32,18 @@ use datafusion::physical_plan::{ use futures::{Stream, StreamExt}; use tracing::warn; +use super::super::block_list::GenMembership; use super::pk::{compute_pk_hash, resolve_pk_indices}; -/// Filters out rows whose PK hash is in any set of `blocked`. +/// Filters out rows whose PK is contained in any newer generation's membership. #[derive(Debug)] pub struct PkHashFilterExec { input: Arc, pk_columns: Vec, - /// Newer generations' membership; a row is blocked if any set holds its hash. - blocked: Vec>>, + /// Newer generations' membership; a row is blocked if any contains its hash. + /// In-memory generations are probed against their index; on-disk ones hold a + /// materialized hash set (see [`GenMembership`]). + blocked: Vec, /// Target neighbor count, used only to warn on a per-source under-fetch. k: usize, properties: Arc, @@ -51,7 +53,7 @@ impl PkHashFilterExec { pub fn new( input: Arc, pk_columns: Vec, - blocked: Vec>>, + blocked: Vec, k: usize, ) -> Self { // A filter preserves the input schema and partitioning. @@ -77,13 +79,11 @@ impl DisplayAs for PkHashFilterExec { DisplayFormatType::Default | DisplayFormatType::Verbose | DisplayFormatType::TreeRender => { - let total: usize = self.blocked.iter().map(|s| s.len()).sum(); write!( f, - "PkHashFilterExec: pk_cols=[{}], gens={}, blocked={}", + "PkHashFilterExec: pk_cols=[{}], gens={}", self.pk_columns.join(", "), self.blocked.len(), - total, ) } } @@ -150,7 +150,7 @@ impl ExecutionPlan for PkHashFilterExec { struct PkHashFilterStream { input: SendableRecordBatchStream, pk_columns: Vec, - blocked: Vec>>, + blocked: Vec, k: usize, schema: SchemaRef, input_seen: usize, @@ -167,7 +167,10 @@ impl PkHashFilterStream { let keep: BooleanArray = (0..batch.num_rows()) .map(|row| { let hash = compute_pk_hash(&batch, &pk_indices, row); - !self.blocked.iter().any(|set| set.contains(&hash)) + !self + .blocked + .iter() + .any(|membership| membership.contains(hash)) }) .collect(); filter_record_batch(&batch, &keep) @@ -223,6 +226,7 @@ mod tests { use datafusion::prelude::SessionContext; use datafusion_physical_plan::test::TestMemoryExec; use futures::TryStreamExt; + use std::collections::HashSet; /// Hash a single-column Int32 PK value the way the exec does, so a test can /// build blocked sets from values rather than hand-computed hashes. @@ -237,8 +241,10 @@ mod tests { RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(ids.to_vec()))]).unwrap() } - fn blocked(ids: &[i32]) -> Vec>> { - vec![Arc::new(ids.iter().map(|&id| hash_int_pk(id)).collect())] + fn blocked(ids: &[i32]) -> Vec { + vec![GenMembership::Set(Arc::new( + ids.iter().map(|&id| hash_int_pk(id)).collect(), + ))] } async fn run(exec: PkHashFilterExec) -> Vec { @@ -275,8 +281,8 @@ mod tests { // Two newer-gen sets: a row is dropped if either contains its PK. let b = int_batch(&[10, 20, 30]); let sets = vec![ - Arc::new(HashSet::from([hash_int_pk(10)])), - Arc::new(HashSet::from([hash_int_pk(30)])), + GenMembership::Set(Arc::new(HashSet::from([hash_int_pk(10)]))), + GenMembership::Set(Arc::new(HashSet::from([hash_int_pk(30)]))), ]; let input = TestMemoryExec::try_new_exec(&[vec![b.clone()]], b.schema(), None).unwrap(); let exec = PkHashFilterExec::new(input, vec!["id".to_string()], sets, 1); @@ -302,11 +308,9 @@ mod tests { let pk = vec!["id".to_string()]; let null_row = with_null(vec![None]); let pk_indices = resolve_pk_indices(&null_row, &pk).unwrap(); - let sets = vec![Arc::new(HashSet::from([compute_pk_hash( - &null_row, - &pk_indices, - 0, - )]))]; + let sets = vec![GenMembership::Set(Arc::new(HashSet::from([ + compute_pk_hash(&null_row, &pk_indices, 0), + ])))]; // Rows: 10, NULL, 30 — only the NULL-key row is dropped. let b = with_null(vec![Some(10), None, Some(30)]); @@ -335,11 +339,9 @@ mod tests { let pk = vec!["id".to_string(), "name".to_string()]; let one_row = mk(&[2], &["b"]); let pk_indices = resolve_pk_indices(&one_row, &pk).unwrap(); - let sets = vec![Arc::new(HashSet::from([compute_pk_hash( - &one_row, - &pk_indices, - 0, - )]))]; + let sets = vec![GenMembership::Set(Arc::new(HashSet::from([ + compute_pk_hash(&one_row, &pk_indices, 0), + ])))]; // (1,"a") and (2,"a") survive; only the exact (2,"b") tuple is dropped. let b = mk(&[1, 2, 2], &["a", "a", "b"]); diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/within_source_dedup.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/within_source_dedup.rs deleted file mode 100644 index be5dae6a668..00000000000 --- a/rust/lance/src/dataset/mem_wal/scanner/exec/within_source_dedup.rs +++ /dev/null @@ -1,432 +0,0 @@ -// SPDX-License-Identifier: Apache-2.0 -// SPDX-FileCopyrightText: Copyright The Lance Authors - -//! WithinSourceDedupExec - Deduplicates rows with the same primary key from a -//! single LSM source, keeping the newest insert. -//! -//! In MemWAL/LSM mode the same primary key can be written multiple times into -//! the same memtable. The active memtable stores rows in insert order (larger -//! `_rowaddr` = newer), while flushed memtables are reverse-written so that -//! within a flushed file the smallest `_rowid` is the newest insert (see -//! `memtable/flush.rs:152` and `hnsw/storage.rs:307`). Point lookup uses this -//! node to collapse such duplicates *within a single source* so that the -//! downstream `CoalesceFirstExec` / `LIMIT` sees at most one row per primary -//! key per source. - -use std::any::Any; -use std::collections::HashMap; -use std::fmt; -use std::pin::Pin; -use std::sync::Arc; -use std::task::{Context, Poll}; - -use arrow_array::{Array, RecordBatch, UInt64Array}; -use arrow_schema::SchemaRef; -use datafusion::error::Result as DFResult; -use datafusion::execution::TaskContext; -use datafusion::physical_expr::{EquivalenceProperties, Partitioning}; -use datafusion::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties, - SendableRecordBatchStream, -}; -use futures::{Stream, StreamExt, ready}; - -use super::pk::{compute_pk_hash, resolve_pk_indices}; - -/// Among rows that share a primary key, which row-address extreme identifies -/// the newest insert to keep. The kept row is always the freshest; only the -/// row address (`_rowaddr`/`_rowid`) used to find it differs by source. -#[derive(Debug, Clone, Copy, PartialEq, Eq)] -pub enum DedupDirection { - /// Keep the row with the largest row-address value (active memtable: larger - /// `_rowaddr` = inserted later). - KeepMaxRowAddr, - /// Keep the row with the smallest row-address value (flushed memtable under - /// reverse-write: smaller `_rowid` = inserted later). - KeepMinRowAddr, -} - -/// Deduplicates rows from a single source by primary key, keeping the row -/// whose `row_addr_column` value wins per [`DedupDirection`]. -/// -/// # Required columns -/// -/// The input must expose: -/// - All `pk_columns` -/// - `row_addr_column` of `UInt64` type -/// -/// The output schema is unchanged from the input. Callers that need to hide -/// the row-address column from downstream consumers should compose this node -/// with `project_to_canonical` or `null_columns`. -/// -/// # Performance -/// -/// Memory: `O(unique primary keys in input)`. For point lookup the input is -/// already filtered to a single primary key so the map holds at most one -/// entry. -#[derive(Debug)] -pub struct WithinSourceDedupExec { - input: Arc, - pk_columns: Vec, - row_addr_column: String, - direction: DedupDirection, - schema: SchemaRef, - properties: Arc, -} - -impl WithinSourceDedupExec { - pub fn new( - input: Arc, - pk_columns: Vec, - row_addr_column: impl Into, - direction: DedupDirection, - ) -> Self { - let schema = input.schema(); - let properties = Arc::new(PlanProperties::new( - EquivalenceProperties::new(schema.clone()), - Partitioning::UnknownPartitioning(1), - input.pipeline_behavior(), - input.boundedness(), - )); - Self { - input, - pk_columns, - row_addr_column: row_addr_column.into(), - direction, - schema, - properties, - } - } - - pub fn pk_columns(&self) -> &[String] { - &self.pk_columns - } - - pub fn row_addr_column(&self) -> &str { - &self.row_addr_column - } - - pub fn direction(&self) -> DedupDirection { - self.direction - } -} - -impl DisplayAs for WithinSourceDedupExec { - fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { - match t { - DisplayFormatType::Default - | DisplayFormatType::Verbose - | DisplayFormatType::TreeRender => { - write!( - f, - "WithinSourceDedupExec: pk=[{}], row_addr={}, direction={:?}", - self.pk_columns.join(", "), - self.row_addr_column, - self.direction, - ) - } - } - } -} - -impl ExecutionPlan for WithinSourceDedupExec { - fn name(&self) -> &str { - "WithinSourceDedupExec" - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - fn properties(&self) -> &Arc { - &self.properties - } - - fn children(&self) -> Vec<&Arc> { - vec![&self.input] - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> DFResult> { - if children.len() != 1 { - return Err(datafusion::error::DataFusionError::Internal( - "WithinSourceDedupExec requires exactly one child".to_string(), - )); - } - Ok(Arc::new(Self::new( - children[0].clone(), - self.pk_columns.clone(), - self.row_addr_column.clone(), - self.direction, - ))) - } - - fn execute( - &self, - partition: usize, - context: Arc, - ) -> DFResult { - let input_stream = self.input.execute(partition, context)?; - Ok(Box::pin(WithinSourceDedupStream { - input: input_stream, - pk_columns: self.pk_columns.clone(), - row_addr_column: self.row_addr_column.clone(), - direction: self.direction, - schema: self.schema.clone(), - winners: HashMap::new(), - emitted: false, - })) - } -} - -/// One winning row, materialized as a single-row `RecordBatch` so we don't -/// have to keep the source batch alive after we've picked the winner. -struct Winner { - batch: RecordBatch, - row_addr: u64, -} - -struct WithinSourceDedupStream { - input: SendableRecordBatchStream, - pk_columns: Vec, - row_addr_column: String, - direction: DedupDirection, - schema: SchemaRef, - winners: HashMap, - emitted: bool, -} - -impl WithinSourceDedupStream { - fn consume_batch(&mut self, batch: RecordBatch) -> DFResult<()> { - if batch.num_rows() == 0 { - return Ok(()); - } - let pk_indices = resolve_pk_indices(&batch, &self.pk_columns)?; - let row_addr_array = batch - .column_by_name(&self.row_addr_column) - .ok_or_else(|| { - datafusion::error::DataFusionError::Internal(format!( - "Row-address column '{}' not found in batch", - self.row_addr_column - )) - })? - .as_any() - .downcast_ref::() - .ok_or_else(|| { - datafusion::error::DataFusionError::Internal(format!( - "Row-address column '{}' is not UInt64", - self.row_addr_column - )) - })?; - - for row_idx in 0..batch.num_rows() { - if row_addr_array.is_null(row_idx) { - // A NULL row address can't be ordered against a real one. Skip - // rather than guess — callers should always project a real - // row-address column for dedup-eligible sources. - continue; - } - let row_addr = row_addr_array.value(row_idx); - let pk_hash = compute_pk_hash(&batch, &pk_indices, row_idx); - - let take_row = match self.winners.get(&pk_hash) { - None => true, - Some(existing) => match self.direction { - DedupDirection::KeepMaxRowAddr => row_addr > existing.row_addr, - DedupDirection::KeepMinRowAddr => row_addr < existing.row_addr, - }, - }; - - if take_row { - let single = batch.slice(row_idx, 1); - self.winners.insert( - pk_hash, - Winner { - batch: single, - row_addr, - }, - ); - } - } - Ok(()) - } - - fn finalize(&mut self) -> DFResult { - if self.winners.is_empty() { - return Ok(RecordBatch::new_empty(self.schema.clone())); - } - let batches: Vec = self.winners.drain().map(|(_, w)| w.batch).collect(); - let batch_refs: Vec<&RecordBatch> = batches.iter().collect(); - arrow_select::concat::concat_batches(&self.schema, batch_refs) - .map_err(|e| datafusion::error::DataFusionError::ArrowError(Box::new(e), None)) - } -} - -impl Stream for WithinSourceDedupStream { - type Item = DFResult; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - loop { - if self.emitted { - return Poll::Ready(None); - } - match ready!(self.input.poll_next_unpin(cx)) { - Some(Ok(batch)) => { - if let Err(e) = self.consume_batch(batch) { - self.emitted = true; - return Poll::Ready(Some(Err(e))); - } - } - Some(Err(e)) => { - self.emitted = true; - return Poll::Ready(Some(Err(e))); - } - None => { - self.emitted = true; - return Poll::Ready(Some(self.finalize())); - } - } - } - } -} - -impl datafusion::physical_plan::RecordBatchStream for WithinSourceDedupStream { - fn schema(&self) -> SchemaRef { - self.schema.clone() - } -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow_array::{Float32Array, Int32Array, StringArray}; - use arrow_schema::{DataType, Field, Schema}; - use datafusion::prelude::SessionContext; - use datafusion_physical_plan::test::TestMemoryExec; - use futures::TryStreamExt; - - fn create_test_schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("name", DataType::Utf8, true), - Field::new("_distance", DataType::Float32, true), - Field::new("_row_addr", DataType::UInt64, true), - ])) - } - - fn batch(ids: &[i32], names: &[&str], distances: &[f32], row_addr: &[u64]) -> RecordBatch { - let schema = create_test_schema(); - RecordBatch::try_new( - schema, - vec![ - Arc::new(Int32Array::from(ids.to_vec())), - Arc::new(StringArray::from(names.to_vec())), - Arc::new(Float32Array::from(distances.to_vec())), - Arc::new(UInt64Array::from(row_addr.to_vec())), - ], - ) - .unwrap() - } - - async fn run(batches: Vec, direction: DedupDirection) -> Vec { - let schema = create_test_schema(); - let input = TestMemoryExec::try_new_exec(&[batches], schema, None).unwrap(); - let exec = - WithinSourceDedupExec::new(input, vec!["id".to_string()], "_row_addr", direction); - let ctx = SessionContext::new(); - let stream = exec.execute(0, ctx.task_ctx()).unwrap(); - stream.try_collect().await.unwrap() - } - - fn extract(batches: &[RecordBatch]) -> Vec<(i32, String, u64)> { - let mut out = Vec::new(); - for b in batches { - let ids = b.column(0).as_any().downcast_ref::().unwrap(); - let names = b.column(1).as_any().downcast_ref::().unwrap(); - let addr = b.column(3).as_any().downcast_ref::().unwrap(); - for i in 0..b.num_rows() { - out.push((ids.value(i), names.value(i).to_string(), addr.value(i))); - } - } - out.sort_by_key(|(id, _, _)| *id); - out - } - - #[tokio::test] - async fn keep_max_picks_largest_row_addr() { - // Active-memtable case: same pk inserted twice; newer = larger _rowaddr. - let b1 = batch( - &[1, 1, 2], - &["old", "new", "two"], - &[0.1, 0.2, 0.3], - &[10, 99, 5], - ); - let out = run(vec![b1], DedupDirection::KeepMaxRowAddr).await; - let rows = extract(&out); - assert_eq!(rows.len(), 2); - assert_eq!(rows[0], (1, "new".to_string(), 99)); - assert_eq!(rows[1], (2, "two".to_string(), 5)); - } - - #[tokio::test] - async fn keep_min_picks_smallest_row_addr() { - // Flushed-memtable case under reverse-write: newer = smaller _rowid. - let b1 = batch( - &[1, 1, 2], - &["old", "new", "two"], - &[0.1, 0.2, 0.3], - &[99, 10, 5], - ); - let out = run(vec![b1], DedupDirection::KeepMinRowAddr).await; - let rows = extract(&out); - assert_eq!(rows.len(), 2); - assert_eq!(rows[0], (1, "new".to_string(), 10)); - assert_eq!(rows[1], (2, "two".to_string(), 5)); - } - - #[tokio::test] - async fn dedup_across_batches() { - let b1 = batch(&[1, 2], &["a", "b"], &[0.1, 0.2], &[1, 1]); - let b2 = batch(&[1, 3], &["a_new", "c"], &[0.5, 0.4], &[7, 1]); - let out = run(vec![b1, b2], DedupDirection::KeepMaxRowAddr).await; - let rows = extract(&out); - assert_eq!(rows.len(), 3); - assert_eq!(rows[0], (1, "a_new".to_string(), 7)); - assert_eq!(rows[1], (2, "b".to_string(), 1)); - assert_eq!(rows[2], (3, "c".to_string(), 1)); - } - - #[tokio::test] - async fn empty_input() { - let out = run(vec![], DedupDirection::KeepMaxRowAddr).await; - let total: usize = out.iter().map(|b| b.num_rows()).sum(); - assert_eq!(total, 0); - } - - #[tokio::test] - async fn null_row_addr_skipped() { - // Rows with NULL row address can't be ordered — they're dropped so they - // don't accidentally become winners against real values. - let schema = create_test_schema(); - let b = RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(Int32Array::from(vec![1, 1])), - Arc::new(StringArray::from(vec!["nulladdr", "real"])), - Arc::new(Float32Array::from(vec![0.1, 0.2])), - Arc::new(UInt64Array::from(vec![None, Some(5)])), - ], - ) - .unwrap(); - let out = run(vec![b], DedupDirection::KeepMaxRowAddr).await; - let rows = extract(&out); - assert_eq!(rows.len(), 1); - assert_eq!(rows[0], (1, "real".to_string(), 5)); - } -} diff --git a/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs b/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs index a6063c2930c..6fc71abd7e7 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs @@ -9,11 +9,14 @@ use std::collections::HashMap; use std::sync::Arc; use arrow_array::{Array, RecordBatch}; -use arrow_schema::SchemaRef; +use arrow_schema::{SchemaRef, SortOptions}; use datafusion::common::ScalarValue; use datafusion::execution::TaskContext; +use datafusion::physical_expr::expressions::Column; +use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_plan::limit::GlobalLimitExec; +use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::prelude::{Expr, SessionContext}; use futures::TryStreamExt; @@ -27,10 +30,7 @@ use crate::dataset::mem_wal::memtable::batch_store::BatchStore; use super::collector::LsmDataSourceCollector; use super::data_source::LsmDataSource; -use super::exec::{ - BloomFilterGuardExec, CoalesceFirstExec, DedupDirection, WithinSourceDedupExec, - compute_pk_hash_from_scalars, -}; +use super::exec::{BloomFilterGuardExec, CoalesceFirstExec, compute_pk_hash_from_scalars}; use super::flushed_cache::{FlushedMemTableCache, open_flushed_dataset}; use super::projection::{ build_scanner_projection, canonical_output_schema, null_columns, project_to_canonical, @@ -573,19 +573,29 @@ impl LsmPointLookupPlanner { // multiple rows sharing the target primary key. scanner.with_row_id(); let raw = scanner.create_plan().await?; - // Within the active memtable, larger `_rowid` = newer - // insert. After dedup there is exactly one row per PK. - let deduped: Arc = Arc::new(WithinSourceDedupExec::new( - raw, - self.pk_columns.clone(), - lance_core::ROW_ID, - DedupDirection::KeepMaxRowAddr, - )); + // The filter already restricts to the exact PK value, so the + // scan yields that key's insert history. Within the active + // memtable larger `_rowid` = newer insert, so sorting `_rowid` + // DESC and keeping the first row picks the newest version — one + // row per (value-exact) PK. + let rowid_idx = raw.schema().index_of(lance_core::ROW_ID)?; + let ordering = LexOrdering::new(vec![PhysicalSortExpr { + expr: Arc::new(Column::new(lance_core::ROW_ID, rowid_idx)), + options: SortOptions { + descending: true, + nulls_first: false, + }, + }]) + .ok_or_else(|| { + lance_core::Error::internal("point-lookup: failed to build _rowid ordering") + })?; + let newest: Arc = + Arc::new(SortExec::new(ordering, raw).with_fetch(Some(1))); // Per-source `_rowid` would collide with the base table's; // NULL it before canonicalization (the value is internal to // this arm). project_to_canonical drops it entirely when // the user didn't request `_rowid` in the projection. - null_columns(deduped, &[lance_core::ROW_ID])? + null_columns(newest, &[lance_core::ROW_ID])? } }; project_to_canonical(scan, &target) @@ -642,10 +652,6 @@ fn probe_position( pk_column: &str, pk_value: &ScalarValue, ) -> Result { - let Some(btree) = index_store.get_btree_by_column(pk_column) else { - return Ok(ProbePos::NoIndex); - }; - // Visible batches are the committed prefix [0, last_visible_idx]; each // `StoredBatch` carries its cumulative `row_offset`, so visibility and the // position→batch mapping are O(1)/O(log) with no per-probe allocation. @@ -661,22 +667,66 @@ fn probe_position( if visible_end == 0 { return Ok(ProbePos::Miss); } + let max_visible_row = visible_end - 1; + + // Prefer a value-keyed scalar BTree on the PK column when one exists: it is + // collision-free, so a single seek-and-stop yields the answer with no value + // re-check. + if let Some(btree) = index_store.get_btree_by_column(pk_column) { + let Some(pos) = btree.get_newest_visible(pk_value, max_visible_row) else { + return Ok(ProbePos::Miss); + }; + let (batch_idx, row) = resolve_position(batch_store, last_visible_idx, pos)?; + return Ok(ProbePos::Found { batch_idx, row }); + } - // Newest visible position of the key — a single seek-and-stop on the - // ordered skiplist (largest key ≤ (value, max_visible_row)). No range - // collect, no allocation. - let Some(pos) = btree.get_newest_visible(pk_value, visible_end - 1) else { - return Ok(ProbePos::Miss); + // Otherwise probe the maintained PK-position index, which is hash-keyed and + // present whenever the table has a primary key (so it covers PK columns + // without their own scalar BTree). Hashing can collide, so verify the + // candidate's actual value: walk this hash's visible positions newest-first + // and return the first whose value equals the query — a colliding key never + // returns the wrong row. The common (no-collision) case checks one position. + let Some(pk_index) = index_store.pk_position_index() else { + return Ok(ProbePos::NoIndex); }; + let hash = compute_pk_hash_from_scalars(std::slice::from_ref(pk_value)); + for pos in pk_index + .visible_positions(hash, max_visible_row) + .into_iter() + .rev() + { + let (batch_idx, row) = resolve_position(batch_store, last_visible_idx, pos)?; + let stored = batch_store + .get(batch_idx) + .ok_or_else(|| lance_core::Error::internal("point-lookup: resolved batch missing"))?; + let col_idx = stored.data.schema_ref().index_of(pk_column).map_err(|_| { + lance_core::Error::internal(format!( + "point-lookup: PK column '{pk_column}' not found in memtable batch" + )) + })?; + let actual = ScalarValue::try_from_array(stored.data.column(col_idx), row)?; + if &actual == pk_value { + return Ok(ProbePos::Found { batch_idx, row }); + } + } + Ok(ProbePos::Miss) +} - // Binary-search the owning batch by `row_offset` (appended in order). +/// Map a global row `position` to its `(batch_idx, row_in_batch)` by binary +/// searching the visible batch prefix on cumulative `row_offset` (batches are +/// appended in order). +fn resolve_position( + batch_store: &BatchStore, + last_visible_idx: usize, + position: u64, +) -> Result<(usize, usize)> { let (mut lo, mut hi) = (0usize, last_visible_idx); while lo < hi { let mid = lo + (hi - lo).div_ceil(2); let off = batch_store.get(mid).map(|b| b.row_offset).ok_or_else(|| { lance_core::Error::internal("point-lookup: batch index out of range during search") })?; - if off <= pos { + if off <= position { lo = mid; } else { hi = mid - 1; @@ -685,10 +735,7 @@ fn probe_position( let stored = batch_store .get(lo) .ok_or_else(|| lance_core::Error::internal("point-lookup: resolved batch missing"))?; - Ok(ProbePos::Found { - batch_idx: lo, - row: (pos - stored.row_offset) as usize, - }) + Ok((lo, (position - stored.row_offset) as usize)) } /// Gather `rows` from `batch_store`'s batch `batch_idx` into the `target` @@ -1097,8 +1144,8 @@ mod tests { // Regression: same primary key inserted twice into one active // memtable must return the *newest* row. The bug was that // `FilterExec → LIMIT 1` over an insert-ordered scan returned the - // first (oldest) match. `WithinSourceDedupExec` collapses by PK, - // keeping the row with the largest `_rowid` (insert order). + // first (oldest) match. The plan-path active arm now sorts `_rowid` + // DESC and keeps the first row (largest `_rowid` = newest insert). use crate::dataset::mem_wal::scanner::collector::{InMemoryMemTableRef, InMemoryMemTables}; use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; use futures::TryStreamExt; @@ -1168,6 +1215,88 @@ mod tests { ); } + #[tokio::test] + async fn test_point_lookup_probes_pk_position_index_without_btree() { + // No scalar BTree on the PK column, only the maintained PK-position + // index (the production default). The fast probe must resolve the newest + // visible version through that index — verifying the value to guard hash + // collisions — rather than falling back to the plan path. + use crate::dataset::mem_wal::scanner::collector::{InMemoryMemTableRef, InMemoryMemTables}; + use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; + + let schema = create_pk_schema(); + let temp_dir = tempfile::tempdir().unwrap(); + let base_uri = format!("{}/base", temp_dir.path().to_str().unwrap()); + + let batch_store = Arc::new(BatchStore::with_capacity(16)); + let mut index_store = IndexStore::new(); + // No `add_btree` — only the PK-position index. + index_store.enable_pk_position_index(vec!["id".to_string()]); + + // pk=1 written twice (the newer second), plus an unrelated pk=2. + let b_old = create_test_batch(&schema, &[1], "old"); + let b_new = create_test_batch(&schema, &[1], "new"); + let b_other = create_test_batch(&schema, &[2], "two"); + let (_, _, bp_old) = batch_store.append(b_old.clone()).unwrap(); + index_store + .insert_with_batch_position(&b_old, 0, Some(bp_old)) + .unwrap(); + let (_, _, bp_new) = batch_store.append(b_new.clone()).unwrap(); + index_store + .insert_with_batch_position(&b_new, 1, Some(bp_new)) + .unwrap(); + let (_, _, bp_other) = batch_store.append(b_other.clone()).unwrap(); + index_store + .insert_with_batch_position(&b_other, 2, Some(bp_other)) + .unwrap(); + let index_store = Arc::new(index_store); + + let shard_id = Uuid::new_v4(); + let collector = LsmDataSourceCollector::without_base_table(base_uri, vec![]) + .with_in_memory_memtables( + shard_id, + InMemoryMemTables { + active: InMemoryMemTableRef { + batch_store, + index_store, + schema: schema.clone(), + generation: 1, + }, + frozen: vec![], + }, + ); + let planner = LsmPointLookupPlanner::new(collector, vec!["id".to_string()], schema); + + // `lookup` takes the fast probe path (single-column PK, no system cols). + let hit = planner + .lookup(&[ScalarValue::Int32(Some(1))], None) + .await + .unwrap() + .expect("pk=1 must be found via the PK-position index probe"); + assert_eq!(hit.num_rows(), 1); + let name = hit + .column_by_name("name") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!( + name.value(0), + "new_1", + "probe must return the newest version" + ); + + // An absent key resolves to None (no on-disk sources to consult). + assert!( + planner + .lookup(&[ScalarValue::Int32(Some(999))], None) + .await + .unwrap() + .is_none(), + "absent key must miss" + ); + } + #[tokio::test] async fn test_point_lookup_flushed_memtable_returns_newest_duplicate() { // Regression / invariant pin: when a flushed memtable contains two diff --git a/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs b/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs index f53d92a3dde..a27b82d9deb 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs @@ -37,10 +37,12 @@ use crate::session::Session; /// Plans vector search queries over LSM data. /// /// Each source is independently newest-per-PK before the union — the active -/// memtable via an over-fetched KNN + within-source dedup, flushed generations -/// via their within-generation deletion vector — and the cross-generation -/// block-list ([`super::exec::PkHashFilterExec`]) drops any PK superseded by a -/// newer generation. So each PK reaches the union from exactly one source and a +/// memtable via an over-fetched KNN + a newest-per-PK recency filter +/// ([`super::exec::NewestPkFilterExec`], which drops a hit that isn't the newest +/// visible version of its PK), flushed generations via their within-generation +/// deletion vector — and the cross-generation block-list +/// ([`super::exec::PkHashFilterExec`]) drops any PK superseded by a newer +/// generation. So each PK reaches the union from exactly one source and a /// distance-ordered merge yields the global top-k; no cross-source dedup is /// needed. /// @@ -53,7 +55,7 @@ use crate::session::Session; /// UnionExec /// ProjectionExec (canonical output schema) /// SortExec(_distance, fetch=k) -/// WithinSourceDedupExec: KeepMaxRowAddr (active) +/// NewestPkFilterExec: newest-per-PK recency (active) /// KNNExec: active memtable, fetch=ceil(k*overfetch) /// ProjectionExec (canonical output schema) /// ProjectionExec (null_columns _rowid) @@ -451,8 +453,8 @@ impl LsmVectorSearchPlanner { build_scanner_projection(projection, &self.base_schema, &self.pk_columns); scanner.project(&cols.iter().map(|s| s.as_str()).collect::>()); // Expose `_rowid` (BatchStore row offset, monotonic with - // insert order) so [`WithinSourceDedupExec`] can collapse - // duplicate-PK rows to the newest insert. The value is + // insert order) so `NewestPkFilterExec` can compare each hit's + // position against the PK-position index. The value is // per-source and NULL'd before reaching the canonical merge. // (VectorIndexExec only plumbs `with_row_id`, not // `with_row_address`, but the two yield identical values @@ -1458,9 +1460,9 @@ mod tests { #[tokio::test] async fn test_vector_search_dedup_within_active_memtable() { // Regression: same PK inserted twice into one active memtable with - // *different* vectors. HNSW indexes each as a distinct node, so - // without WithinSourceDedupExec a KNN can return both candidates - // for the same PK and pollute top-k. The newer insert must win. + // *different* vectors. HNSW indexes each as a distinct node, so without + // the recency filter a KNN can return both candidates for the same PK + // and pollute top-k. The newer insert must win. use crate::dataset::mem_wal::scanner::collector::{InMemoryMemTableRef, InMemoryMemTables}; use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; use datafusion::prelude::SessionContext; @@ -1578,8 +1580,8 @@ mod tests { // // Within a *single* active memtable, pk=1 is first inserted ON the query // (distance ~0), then updated to a FAR vector. The append-only HNSW keeps - // both nodes live. WithinSourceDedupExec(KeepMaxRowAddr) only collapses - // duplicate PKs that are BOTH present in the over-fetched candidate set. + // both nodes live. A result-set dedup only collapses duplicate PKs that + // are BOTH present in the over-fetched candidate set. // // Here the fresh (far) pk=1 is evicted from the candidate set — there are // enough nearer filler rows that it ranks below the fetch cutoff — so the From 64d97e2cdbe8b02a8e3bba4b96dc9174942843e9 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 2 Jun 2026 21:11:06 -0500 Subject: [PATCH 04/12] refactor(mem_wal): probe the PK-position index in contains_pks; delete pk_hashes() `fresh_tier_block_list` now returns `Vec` instead of `Vec>>`, and `LsmScanner::contains_pks` tests each PK by probing the membership (`GenMembership::contains`) rather than against a materialized set. In-memory tiers are probed against their PK-position index (snapshot-bounded); flushed/base keep their cached hash sets. This retires the last per-query PK-hash set built from an in-memory memtable: `PkPositionIndex::pk_hashes()` and `block_list::in_memory_pk_hashes` are deleted. `pk_hashes_from_batch_store` remains only as the no-PK-index fallback. `contains_pks`'s public signature (&RecordBatch -> Vec) is unchanged. Co-Authored-By: Claude Opus 4.8 (1M context) --- .../src/dataset/mem_wal/index/pk_position.rs | 8 -- .../src/dataset/mem_wal/scanner/block_list.rs | 74 ++++++++----------- .../src/dataset/mem_wal/scanner/builder.rs | 4 +- 3 files changed, 31 insertions(+), 55 deletions(-) diff --git a/rust/lance/src/dataset/mem_wal/index/pk_position.rs b/rust/lance/src/dataset/mem_wal/index/pk_position.rs index eff1e7638d3..1a78e12ada1 100644 --- a/rust/lance/src/dataset/mem_wal/index/pk_position.rs +++ b/rust/lance/src/dataset/mem_wal/index/pk_position.rs @@ -20,7 +20,6 @@ //! `max_visible` watermark is unaffected by concurrent appends (which only add //! larger positions), so no snapshot needs to be co-published with the query. -use std::collections::HashSet; use std::sync::Mutex; use arrow_array::RecordBatch; @@ -140,13 +139,6 @@ impl PkPositionIndex { positions } - /// Every distinct primary-key hash currently in the index. The cross-source - /// block-list uses this as an in-memory generation's membership set instead - /// of re-scanning (and re-hashing) the `BatchStore` per query. - pub fn pk_hashes(&self) -> HashSet { - self.reader.iter().map(|key| key.hash).collect() - } - /// Number of entries (one per inserted row, not per distinct key). pub fn len(&self) -> usize { self.reader.len() diff --git a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs index b9edcc00d0d..4483827fec8 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs @@ -3,11 +3,13 @@ //! Per-source block-list construction for LSM vector search. //! -//! A generation's membership is an `Arc>` of PK hashes -//! ([`compute_pk_hash`]), built once (immutable gens cached). Each source gets a -//! `Vec>>` of the newer generations' sets (`NEWER(G)`; base: all -//! of them) — referenced, never merged. The KNN drops candidates whose PK is in -//! any (see [`super::exec::PkHashFilterExec`]). +//! A generation's membership is a [`GenMembership`]: in-memory generations +//! (active / frozen) are **probed** against their maintained PK-position index +//! (no per-query set), while on-disk generations (flushed, base) carry a cached +//! `Arc>` of PK hashes ([`compute_pk_hash`]). Each source gets a +//! `Vec` of the newer generations (`NEWER(G)`; base: all of +//! them); the KNN drops a candidate whose PK is in any (see +//! [`super::exec::PkHashFilterExec`]). //! //! Cross-generation only: within-gen dups share a hash and fall to the global //! dedup's `(generation, freshness)` tiebreaker. @@ -162,54 +164,36 @@ pub async fn compute_source_block_lists( Ok(blocked) } -/// The fresh-tier block-list: one membership set per generation that shadows the -/// base table — active + frozen memtables (hashed now) and flushed generations -/// (from the cache). Same `Vec>>` shape the vector-search filter -/// consumes; a base/external reader can drop any row whose PK is in one of them. -/// The base source, if present, is skipped (it is what gets shadowed). +/// The fresh-tier block-list: one [`GenMembership`] per generation that shadows +/// the base table — active + frozen memtables (probed against their index) and +/// flushed generations (cached PK-hash sets). A base/external reader can test +/// any PK against these (e.g. via `contains`) to decide whether the fresh tier +/// shadows it. The base source, if present, is skipped (it is what gets +/// shadowed). pub async fn fresh_tier_block_list( sources: &[LsmDataSource], pk_columns: &[String], session: Option<&Arc>, flushed_cache: Option<&Arc>, -) -> Result>>> { - let mut sets = Vec::new(); +) -> Result> { + let mut memberships = Vec::new(); for source in sources { - let set = match source { + let membership = match source { LsmDataSource::BaseTable { .. } => continue, LsmDataSource::ActiveMemTable { batch_store, index_store, .. - } => Arc::new(in_memory_pk_hashes(batch_store, index_store, pk_columns)?), - LsmDataSource::FlushedMemTable { path, .. } => { - flushed_pk_hashes(path, pk_columns, session, flushed_cache).await? - } + } => in_memory_membership(batch_store, index_store, pk_columns)?, + LsmDataSource::FlushedMemTable { path, .. } => GenMembership::Set( + flushed_pk_hashes(path, pk_columns, session, flushed_cache).await?, + ), }; - if !set.is_empty() { - sets.push(set); + if !membership.is_empty() { + memberships.push(membership); } } - Ok(sets) -} - -/// PK-hash membership of an in-memory (active / frozen) memtable. -/// -/// Reads it straight from the memtable's maintained MVCC PK-position index — its -/// keyset *is* the membership set, already hashed, so there is nothing to -/// re-scan or re-hash. Falls back to a one-time `BatchStore` scan only when the -/// memtable has no PK-position index (e.g. a table without a primary key), which -/// the production vector-search path never hits since that index is always -/// enabled alongside the secondary indexes. -fn in_memory_pk_hashes( - batch_store: &BatchStore, - index_store: &IndexStore, - pk_columns: &[String], -) -> Result> { - match index_store.pk_position_index() { - Some(index) => Ok(index.pk_hashes()), - None => pk_hashes_from_batch_store(batch_store, pk_columns), - } + Ok(memberships) } /// Cross-source membership of an in-memory (active / frozen) memtable. @@ -402,17 +386,17 @@ mod tests { // Active gen 2: pk=1,2. Frozen gen 1: pk=3. let sources = vec![mk(&[1, 2], 2), mk(&[3], 1)]; - let sets = fresh_tier_block_list(&sources, &["id".to_string()], None, None) + let memberships = fresh_tier_block_list(&sources, &["id".to_string()], None, None) .await .unwrap(); - // One set per generation; together they cover pk=1,2,3 (not 4). - assert_eq!(sets.len(), 2); - let set_blocks = |id: i32| sets.iter().any(|s| s.contains(&hash_id(id))); + // One membership per generation; together they cover pk=1,2,3 (not 4). + assert_eq!(memberships.len(), 2); + let fresh_blocks = |id: i32| memberships.iter().any(|m| m.contains(hash_id(id))); for id in [1, 2, 3] { - assert!(set_blocks(id)); + assert!(fresh_blocks(id)); } - assert!(!set_blocks(4)); + assert!(!fresh_blocks(4)); } #[tokio::test] diff --git a/rust/lance/src/dataset/mem_wal/scanner/builder.rs b/rust/lance/src/dataset/mem_wal/scanner/builder.rs index ade4164d485..59149ef93ba 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/builder.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/builder.rs @@ -457,7 +457,7 @@ impl LsmScanner { /// [`FlushedMemTableCache`] when one is set. pub async fn contains_pks(&self, pks: &RecordBatch) -> Result> { let sources = self.build_collector().collect()?; - let sets = super::block_list::fresh_tier_block_list( + let memberships = super::block_list::fresh_tier_block_list( &sources, &self.pk_columns, self.session.as_ref(), @@ -469,7 +469,7 @@ impl LsmScanner { Ok((0..pks.num_rows()) .map(|row| { let hash = super::exec::compute_pk_hash(pks, &pk_indices, row); - sets.iter().any(|set| set.contains(&hash)) + memberships.iter().any(|m| m.contains(hash)) }) .collect()) } From 648cf6062da92c7cc6b1d536674195a43772d706 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Tue, 2 Jun 2026 21:49:11 -0500 Subject: [PATCH 05/12] test(mem_wal): cover the snapshot-bounded vanished-row guards MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adds the two correctness cases the index probing introduces: - NewestPkFilterExec (within-source): keeps only the newest visible position per PK, and — crucially — does NOT vanish a visible row when a newer version was committed after the query's snapshot (max_visible bounds the seek). Also covers the no-PK-index pass-through. - block_list (cross-source): index-sourced membership is snapshot-bounded, so a newer generation's not-yet-visible write does not shadow an older generation's visible copy (while a visible newer write still does). Co-Authored-By: Claude Opus 4.8 (1M context) --- .../src/dataset/mem_wal/scanner/block_list.rs | 69 ++++++++++ .../mem_wal/scanner/exec/newest_pk_filter.rs | 125 ++++++++++++++++++ 2 files changed, 194 insertions(+) diff --git a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs index 4483827fec8..ed6d5558630 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs @@ -615,4 +615,73 @@ mod tests { assert!(blocks(&blocked[&(Some(shard), g1)], 2)); assert!(!blocked.contains_key(&(Some(shard), LsmGeneration::memtable(2)))); } + + #[tokio::test] + async fn index_membership_is_snapshot_bounded() { + // The index-sourced membership only counts a PK whose version is visible + // at the source's watermark, so a newer generation's not-yet-visible + // write can't shadow an older generation's visible copy. + use crate::dataset::mem_wal::scanner::data_source::{LsmDataSource, LsmGeneration}; + use crate::dataset::mem_wal::write::IndexStore; + use uuid::Uuid; + + let shard = Uuid::new_v4(); + let schema = id_batch(&[1]).schema(); + + // Older frozen gen 1: pk=1 (no PK-position index → Set fallback). + let g1_store = BatchStore::with_capacity(8); + g1_store.append(id_batch(&[1])).unwrap(); + + // Newer active gen 2: pk=99 visible at position 0, then pk=1 written at + // position 1 but with the watermark left at batch 0 (so pk=1 is in the + // index yet not visible) — the concurrent-write race. + let g2_store = BatchStore::with_capacity(8); + let mut g2_index = IndexStore::new(); + g2_index.enable_pk_position_index(vec!["id".to_string()]); + let b0 = id_batch(&[99]); + let (bp0, off0, _) = g2_store.append(b0.clone()).unwrap(); + g2_index + .insert_with_batch_position(&b0, off0, Some(bp0)) // advances watermark to 0 + .unwrap(); + let b1 = id_batch(&[1]); + let (_, off1, _) = g2_store.append(b1.clone()).unwrap(); + g2_index + .insert_with_batch_position(&b1, off1, None) // index updated, watermark unchanged + .unwrap(); + + let sources = vec![ + LsmDataSource::ActiveMemTable { + batch_store: Arc::new(g1_store), + index_store: Arc::new(IndexStore::new()), + schema: schema.clone(), + shard_id: shard, + generation: LsmGeneration::memtable(1), + }, + LsmDataSource::ActiveMemTable { + batch_store: Arc::new(g2_store), + index_store: Arc::new(g2_index), + schema, + shard_id: shard, + generation: LsmGeneration::memtable(2), + }, + ]; + + let blocked = Box::pin(compute_source_block_lists( + &sources, + &["id".to_string()], + None, + None, + )) + .await + .unwrap(); + + let g1_block = &blocked[&(Some(shard), LsmGeneration::memtable(1))]; + // pk=99 is visible in gen 2 → it blocks gen 1's pk=99. + assert!(blocks(g1_block, 99)); + // pk=1's only gen-2 copy is not yet visible → it must NOT shadow gen 1. + assert!( + !blocks(g1_block, 1), + "a not-yet-visible newer write must not shadow an older visible copy" + ); + } } diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs index 6ca39c56413..a9ad14856c4 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs @@ -258,3 +258,128 @@ impl datafusion::physical_plan::RecordBatchStream for NewestPkFilterStream { self.schema.clone() } } + +#[cfg(test)] +mod tests { + use super::*; + use arrow_array::Int32Array; + use arrow_schema::{DataType, Field, Schema}; + use datafusion::prelude::SessionContext; + use datafusion_physical_plan::test::TestMemoryExec; + use futures::TryStreamExt; + + /// Single-column `id` PK batch, one per append so a caller can control + /// row-level visibility via `max_visible_batch_position`. + fn id_batch(id: i32) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(vec![id]))]).unwrap() + } + + /// Index-search "hits": `(id, _rowid)` pairs the filter evaluates. + fn hits(rows: &[(i32, u64)]) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new(lance_core::ROW_ID, DataType::UInt64, true), + ])); + let ids: Vec = rows.iter().map(|(id, _)| *id).collect(); + let rowids: Vec = rows.iter().map(|(_, p)| *p).collect(); + RecordBatch::try_new( + schema, + vec![ + Arc::new(Int32Array::from(ids)), + Arc::new(UInt64Array::from(rowids)), + ], + ) + .unwrap() + } + + /// Build an active memtable whose PK-position index + BatchStore hold one + /// row per `id` in `appended` (positions 0..n), all committed. + fn active(appended: &[i32]) -> (Arc, Arc) { + let batch_store = Arc::new(BatchStore::with_capacity(16)); + let mut index = IndexStore::new(); + index.enable_pk_position_index(vec!["id".to_string()]); + for &id in appended { + let b = id_batch(id); + let (bp, off, _) = batch_store.append(b.clone()).unwrap(); + index.insert_with_batch_position(&b, off, Some(bp)).unwrap(); + } + (Arc::new(index), batch_store) + } + + async fn run( + index_store: Arc, + batch_store: Arc, + max_visible_batch_position: usize, + hits_batch: RecordBatch, + ) -> Vec<(i32, u64)> { + let input = + TestMemoryExec::try_new_exec(&[vec![hits_batch.clone()]], hits_batch.schema(), None) + .unwrap(); + let exec = NewestPkFilterExec::new( + input, + vec!["id".to_string()], + lance_core::ROW_ID, + index_store, + batch_store, + max_visible_batch_position, + ); + let ctx = SessionContext::new(); + let out: Vec = exec + .execute(0, ctx.task_ctx()) + .unwrap() + .try_collect() + .await + .unwrap(); + let mut rows = Vec::new(); + for b in &out { + let ids = b.column(0).as_any().downcast_ref::().unwrap(); + let pos = b.column(1).as_any().downcast_ref::().unwrap(); + for i in 0..b.num_rows() { + rows.push((ids.value(i), pos.value(i))); + } + } + rows + } + + #[tokio::test] + async fn keeps_only_the_newest_visible_position_per_pk() { + // id=1 written at positions 0 and 2 (an update), id=2 at position 1; all + // visible. A stale hit (id=1 @ 0) is dropped; the newest (id=1 @ 2) and + // the unrelated id=2 survive — even though all three were "returned" by + // the index search. + let (index, store) = active(&[1, 2, 1]); + let rows = run(index, store, 2, hits(&[(1, 0), (2, 1), (1, 2)])).await; + assert_eq!(rows, vec![(2, 1), (1, 2)]); + } + + #[tokio::test] + async fn does_not_vanish_a_visible_row_under_a_newer_invisible_write() { + // The store/index hold id=1 at positions 0 and 2, but the query latched + // `max_visible_batch_position = 0` (only position 0 visible) — i.e. the + // update at position 2 was committed *after* this query's snapshot. The + // visible older row (id=1 @ 0) must be KEPT (its newest *visible* version + // is itself), not dropped because of the not-yet-visible position 2. + let (index, store) = active(&[1, 2, 1]); + let kept = run(index.clone(), store.clone(), 0, hits(&[(1, 0)])).await; + assert_eq!(kept, vec![(1, 0)], "visible row must not vanish"); + + // And the not-yet-visible position is itself dropped (outside snapshot). + let dropped = run(index, store, 0, hits(&[(1, 2)])).await; + assert!( + dropped.is_empty(), + "row beyond the snapshot must be dropped" + ); + } + + #[tokio::test] + async fn passes_through_when_no_pk_position_index() { + // A memtable without a PK-position index can't be deduped here, so the + // filter is a pass-through rather than dropping everything. + let batch_store = Arc::new(BatchStore::with_capacity(16)); + batch_store.append(id_batch(1)).unwrap(); + let index = Arc::new(IndexStore::new()); // no enable_pk_position_index + let rows = run(index, batch_store, 0, hits(&[(1, 0), (1, 9)])).await; + assert_eq!(rows, vec![(1, 0), (1, 9)]); + } +} From 14428ffb58829c93c5e93d834223c8ead4edac62 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Wed, 3 Jun 2026 13:31:47 -0500 Subject: [PATCH 06/12] refactor(mem_wal): reuse PK BTree indexes for dedup; drop PkPositionIndex MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Maintain a BTree index on each primary-key column (reusing a user's scalar index when present, else auto-creating an in-memory `__pk__*` one) and compose them for tuple-level MVCC lookups, replacing the separate hash-keyed PkPositionIndex. Because row position is the physical row identity, a composite key's newest visible row is the largest position present in every column's BTree at or below the watermark — collision-free and uniform for single and composite PKs. - index/pk_lookup.rs: new PkLookup over the per-column PK BTrees (get_newest_visible / is_newest / contains_visible). - BTreeMemIndex: add visible_positions + contains_position primitives. - IndexStore: btree_indexes now Arc-valued; enable_pk_index reuses or auto-creates PK BTrees (maintained by the existing insert loop, in-memory only — flush still rebuilds from user index configs); pk_index() -> PkLookup. - NewestPkFilterExec / point_lookup / block_list now probe by value; point_lookup's hash fallback + re-check deleted. Block-list flushed/base arm stays hash-keyed; needs_values() skips per-row value extraction otherwise. - Delete index/pk_position.rs. Addresses jackye1995's review on #7067. Sharp edge: composite-PK lookups intersect per-column position lists, so put the most selective column first. cargo test -p lance --lib mem_wal: 385 passed; fmt + clippy clean. Co-Authored-By: Claude Opus 4.8 --- rust/lance/src/dataset/mem_wal/index.rs | 109 +++++---- rust/lance/src/dataset/mem_wal/index/btree.rs | 107 ++++++++ .../src/dataset/mem_wal/index/pk_lookup.rs | 186 ++++++++++++++ .../src/dataset/mem_wal/index/pk_position.rs | 228 ------------------ .../src/dataset/mem_wal/scanner/block_list.rs | 67 ++--- .../src/dataset/mem_wal/scanner/builder.rs | 26 +- .../mem_wal/scanner/exec/newest_pk_filter.rs | 66 ++--- .../mem_wal/scanner/exec/pk_hash_filter.rs | 28 ++- .../src/dataset/mem_wal/scanner/fts_search.rs | 6 +- .../dataset/mem_wal/scanner/point_lookup.rs | 65 ++--- .../dataset/mem_wal/scanner/vector_search.rs | 24 +- rust/lance/src/dataset/mem_wal/write.rs | 14 +- 12 files changed, 519 insertions(+), 407 deletions(-) create mode 100644 rust/lance/src/dataset/mem_wal/index/pk_lookup.rs delete mode 100644 rust/lance/src/dataset/mem_wal/index/pk_position.rs diff --git a/rust/lance/src/dataset/mem_wal/index.rs b/rust/lance/src/dataset/mem_wal/index.rs index dff7525bc5d..4563f33e00c 100644 --- a/rust/lance/src/dataset/mem_wal/index.rs +++ b/rust/lance/src/dataset/mem_wal/index.rs @@ -18,9 +18,10 @@ mod arena_skiplist; mod btree; mod fts; mod hnsw; -mod pk_position; +mod pk_lookup; use std::collections::HashMap; +use std::sync::Arc; use std::sync::atomic::{AtomicUsize, Ordering}; use super::memtable::batch_store::StoredBatch; @@ -45,7 +46,7 @@ pub type RowPosition = u64; pub use btree::{BTreeIndexConfig, BTreeMemIndex}; pub use fts::{FtsIndexConfig, FtsMemIndex, FtsQueryExpr, SearchOptions}; pub use hnsw::{HnswIndexConfig, HnswMemIndex}; -pub use pk_position::PkPositionIndex; +pub use pk_lookup::PkLookup; // ============================================================================ // Index Store @@ -197,17 +198,20 @@ impl MemIndexConfig { /// therefore safe for scanners to read. Scanners snapshot this at plan /// construction time so every plan keys on a stable MVCC cursor. pub struct IndexStore { - /// BTree indexes keyed by index name. - btree_indexes: HashMap, + /// BTree indexes keyed by index name. `Arc` so the primary-key BTrees can be + /// shared into [`Self::pk_btrees`] without a second copy or a second insert. + btree_indexes: HashMap>, /// HNSW vector indexes keyed by index name. hnsw_indexes: HashMap, /// FTS indexes keyed by index name. fts_indexes: HashMap, - /// MVCC primary-key → newest-position index. Present when the memtable has a - /// primary key; maintained on every insert and read by the active vector / - /// FTS arms to drop predicate-crossing stale rows. Not an `Option<&str>`-keyed - /// map like the others — there is at most one per memtable. - pk_position_index: Option, + /// The primary-key BTrees, one per PK column in primary-key order. Each + /// aliases an entry in `btree_indexes` (a reused user index, or an + /// auto-created `__pk__*` one), so it is maintained by the same insert loop. + /// Empty when the memtable has no primary key. Read as a [`PkLookup`] by the + /// active vector / FTS arms and the cross-source block-list to find the + /// newest visible version of a key (see [`Self::enable_pk_index`]). + pk_btrees: Vec>, /// Maximum batch position that is durable in the WAL and therefore /// visible to scanners. Advanced unconditionally after a WAL append /// succeeds; not gated on whether any indexes are configured. @@ -220,7 +224,7 @@ impl Default for IndexStore { btree_indexes: HashMap::new(), hnsw_indexes: HashMap::new(), fts_indexes: HashMap::new(), - pk_position_index: None, + pk_btrees: Vec::new(), max_visible_batch_position: AtomicUsize::new(0), } } @@ -238,7 +242,14 @@ impl std::fmt::Debug for IndexStore { &self.hnsw_indexes.keys().collect::>(), ) .field("fts_indexes", &self.fts_indexes.keys().collect::>()) - .field("pk_position_index", &self.pk_position_index) + .field( + "pk_btrees", + &self + .pk_btrees + .iter() + .map(|b| b.column_name()) + .collect::>(), + ) .field( "max_visible_batch_position", &self.max_visible_batch_position.load(Ordering::Acquire), @@ -273,7 +284,7 @@ impl IndexStore { for config in configs { match config { MemIndexConfig::BTree(c) => { - let index = BTreeMemIndex::new(c.field_id, c.column.clone()); + let index = Arc::new(BTreeMemIndex::new(c.field_id, c.column.clone())); registry.btree_indexes.insert(c.name.clone(), index); } MemIndexConfig::Hnsw(c) => { @@ -302,7 +313,7 @@ impl IndexStore { /// the production memtable path goes through [`Self::from_configs`]. pub fn add_btree(&mut self, name: String, field_id: i32, column: String) { self.btree_indexes - .insert(name, BTreeMemIndex::new(field_id, column)); + .insert(name, Arc::new(BTreeMemIndex::new(field_id, column))); } /// Add an HNSW vector index with default build parameters. @@ -371,22 +382,42 @@ impl IndexStore { .insert(name, FtsMemIndex::with_params(field_id, column, params)); } - /// Enable the MVCC primary-key → newest-position index over `pk_columns`. + /// Maintain a BTree index on each primary-key column so the memtable can + /// answer "newest visible version of this key" (see [`Self::pk_index`]). + /// + /// For each `(column, field_id)` an existing BTree on that field is reused; + /// otherwise an auto-created one is registered under a synthetic `__pk__*` + /// name so the normal insert loop maintains it. Call this once at + /// construction, after [`Self::from_configs`] (so user indexes are present + /// to reuse) and before any inserts. A no-op when `pk_columns` is empty (a + /// memtable without a primary key has nothing to dedup). /// - /// Build this once at construction, before any inserts (it is maintained on - /// every subsequent insert). A no-op when `pk_columns` is empty (a memtable - /// without a primary key has nothing to dedup). The active vector / FTS - /// search arms read it via [`Self::pk_position_index`] to drop stale rows. - pub fn enable_pk_position_index(&mut self, pk_columns: Vec) { - if !pk_columns.is_empty() { - self.pk_position_index = Some(PkPositionIndex::new(pk_columns)); + /// The auto-created BTrees are in-memory only — flush rebuilds indexes from + /// the user's index configs, not from this map — so a primary key without a + /// user-defined scalar index does not gain an on-disk index. + pub fn enable_pk_index(&mut self, pk_columns: &[(String, i32)]) { + for (column, field_id) in pk_columns { + let btree = match self + .btree_indexes + .values() + .find(|b| b.field_id() == *field_id) + { + Some(existing) => existing.clone(), + None => { + let btree = Arc::new(BTreeMemIndex::new(*field_id, column.clone())); + self.btree_indexes + .insert(format!("__pk__{column}"), btree.clone()); + btree + } + }; + self.pk_btrees.push(btree); } } - /// The MVCC primary-key → newest-position index, if the memtable has a - /// primary key (see [`Self::enable_pk_position_index`]). - pub fn pk_position_index(&self) -> Option<&PkPositionIndex> { - self.pk_position_index.as_ref() + /// A tuple-level view over the primary-key BTrees, if the memtable has a + /// primary key (see [`Self::enable_pk_index`]). + pub fn pk_index(&self) -> Option> { + (!self.pk_btrees.is_empty()).then(|| PkLookup::new(&self.pk_btrees)) } /// Insert a batch into all indexes. @@ -411,9 +442,8 @@ impl IndexStore { for index in self.fts_indexes.values() { index.insert(batch, row_offset)?; } - if let Some(pk_index) = &self.pk_position_index { - pk_index.insert(batch, row_offset)?; - } + // The primary-key BTrees are aliases of `btree_indexes` entries, so the + // loop above already maintained them. // Update global watermark after all indexes have been updated if let Some(bp) = batch_position { @@ -470,12 +500,8 @@ impl IndexStore { } } - // PK-position index: one entry per row, in batch order. - if let Some(pk_index) = &self.pk_position_index { - for stored in batches { - pk_index.insert(&stored.data, stored.row_offset)?; - } - } + // The primary-key BTrees are aliases of `btree_indexes` entries, so the + // loop above already maintained them. // Update global watermark to the max batch position let max_bp = batches.iter().map(|b| b.batch_position).max().unwrap(); @@ -589,14 +615,9 @@ impl IndexStore { .map(|(name, _idx_type, duration)| (name.to_string(), duration)) .collect(); - // PK-position index: cheap (one skiplist insert per row), updated - // inline after the parallel index threads join so it is in place - // before the visibility watermark advances below. - if let Some(pk_index) = &self.pk_position_index { - for stored in batches { - pk_index.insert(&stored.data, stored.row_offset)?; - } - } + // The primary-key BTrees are aliases of `btree_indexes` entries, so + // their per-index threads above already maintained them (and joined + // before the visibility watermark advances below). // Update global watermark to the max batch position let max_bp = batches.iter().map(|b| b.batch_position).max().unwrap(); @@ -608,7 +629,7 @@ impl IndexStore { /// Get a BTree index by name. pub fn get_btree(&self, name: &str) -> Option<&BTreeMemIndex> { - self.btree_indexes.get(name) + self.btree_indexes.get(name).map(Arc::as_ref) } /// Get an HNSW vector index by name. @@ -629,6 +650,7 @@ impl IndexStore { self.btree_indexes .values() .find(|idx| idx.field_id() == field_id) + .map(Arc::as_ref) } /// Get an HNSW vector index by field ID. @@ -653,6 +675,7 @@ impl IndexStore { self.btree_indexes .values() .find(|idx| idx.column_name() == column) + .map(Arc::as_ref) } /// Get an HNSW vector index by column name. diff --git a/rust/lance/src/dataset/mem_wal/index/btree.rs b/rust/lance/src/dataset/mem_wal/index/btree.rs index c2f89b9932d..30145db7293 100644 --- a/rust/lance/src/dataset/mem_wal/index/btree.rs +++ b/rust/lance/src/dataset/mem_wal/index/btree.rs @@ -398,6 +398,35 @@ impl FixedIntBackend { positions } + fn visible_positions(&self, value: &ScalarValue, max: RowPosition) -> Vec { + let Some(enc) = encode_scalar(value) else { + if value.is_null() { + let mut positions: Vec = self + .null_positions + .lock() + .unwrap() + .iter() + .copied() + .filter(|p| *p <= max) + .collect(); + positions.sort_unstable(); + return positions; + } + return Vec::new(); + }; + let start = FixedKey { enc, position: 0 }; + let mut positions = Vec::new(); + // Keys are ordered by (enc, position); for a fixed enc, positions are + // ascending, so the first position past the watermark ends the walk. + for key in self.reader.range_from(&start) { + if key.enc != enc || key.position > max { + break; + } + positions.push(key.position); + } + positions + } + fn len(&self) -> usize { self.reader.len() + self.null_positions.lock().unwrap().len() } @@ -554,6 +583,36 @@ impl BytesBackend { positions } + fn visible_positions(&self, value: &ScalarValue, max: RowPosition) -> Vec { + let Some(bytes) = value_bytes(value) else { + if value.is_null() { + let mut positions: Vec = self + .null_positions + .lock() + .unwrap() + .iter() + .copied() + .filter(|p| *p <= max) + .collect(); + positions.sort_unstable(); + return positions; + } + return Vec::new(); + }; + let start = BytesKey { + bytes: InlineBytes::new(bytes), + position: 0, + }; + let mut positions = Vec::new(); + for key in self.reader.range_from(&start) { + if key.bytes.as_slice() != bytes || key.position > max { + break; + } + positions.push(key.position); + } + positions + } + fn len(&self) -> usize { self.reader.len() + self.null_positions.lock().unwrap().len() } @@ -728,6 +787,21 @@ impl ScalarBackend { positions } + fn visible_positions(&self, value: &ScalarValue, max: RowPosition) -> Vec { + let start = IndexKey { + value: OrderableScalarValue(value.clone()), + row_position: 0, + }; + let mut positions = Vec::new(); + for key in self.reader.range_from(&start) { + if key.value.0 != *value || key.row_position > max { + break; + } + positions.push(key.row_position); + } + positions + } + fn len(&self) -> usize { self.reader.len() } @@ -793,6 +867,14 @@ impl Backend { } } + fn visible_positions(&self, value: &ScalarValue, max: RowPosition) -> Vec { + match self { + Self::FixedInt(b) => b.visible_positions(value, max), + Self::Bytes(b) => b.visible_positions(value, max), + Self::Scalar(b) => b.visible_positions(value, max), + } + } + fn len(&self) -> usize { match self { Self::FixedInt(b) => b.len(), @@ -893,6 +975,31 @@ impl BTreeMemIndex { self.backend.get().map(|b| b.get(value)).unwrap_or_default() } + /// All row positions for `value` that are `<= max_visible_row`, ascending. + /// + /// The composite-primary-key driver: a tuple's newest visible row is found + /// by walking the leading column's visible positions newest-first and + /// keeping the first that every other column also holds at the same physical + /// position (see [`super::PkLookup`]). + pub fn visible_positions( + &self, + value: &ScalarValue, + max_visible_row: RowPosition, + ) -> Vec { + self.backend + .get() + .map(|b| b.visible_positions(value, max_visible_row)) + .unwrap_or_default() + } + + /// Whether the exact `(value, position)` entry exists. One seek-and-stop: + /// the newest entry for `value` at or below `position` is `position` itself + /// iff that entry is present. Used to test a candidate row position against + /// the other primary-key columns when composing a composite key. + pub fn contains_position(&self, value: &ScalarValue, position: RowPosition) -> bool { + self.get_newest_visible(value, position) == Some(position) + } + /// Get the number of entries (not unique values). pub fn len(&self) -> usize { self.backend.get().map(|b| b.len()).unwrap_or(0) diff --git a/rust/lance/src/dataset/mem_wal/index/pk_lookup.rs b/rust/lance/src/dataset/mem_wal/index/pk_lookup.rs new file mode 100644 index 00000000000..f04e28d89c9 --- /dev/null +++ b/rust/lance/src/dataset/mem_wal/index/pk_lookup.rs @@ -0,0 +1,186 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright The Lance Authors + +//! Tuple-level MVCC primary-key lookups over the per-column BTree indexes. +//! +//! The memtable maintains a [`super::BTreeMemIndex`] on every primary-key +//! column (reusing a user's scalar index when one exists, otherwise an +//! auto-created one — see [`super::IndexStore::enable_pk_index`]). Each BTree is +//! value-keyed as `(value, row_position)`, so for one column +//! [`super::BTreeMemIndex::get_newest_visible`] answers "the newest row position +//! for this value visible at the watermark" with a single seek. +//! +//! [`PkLookup`] composes those per-column answers into a single primary-key +//! question. Because `row_position` is the physical row identity, the rows +//! matching a tuple `(v0 … vn)` are exactly the intersection of each column's +//! position set, so the newest visible row of the tuple is the largest position +//! present in *every* column at or below the watermark. This is collision-free +//! (no hashing) and covers single and composite primary keys uniformly. +//! +//! Cost: a single-column primary key is one seek. A composite key walks the +//! leading column's positions for `v0` and point-probes the rest, so its cost +//! tracks the leading column's cardinality for that value — put the most +//! selective column first in a composite primary key. + +use std::sync::Arc; + +use datafusion::common::ScalarValue; + +use super::RowPosition; +use super::btree::BTreeMemIndex; + +/// A read-only view over the primary-key BTree indexes (one per PK column, in +/// primary-key order), composing them into tuple-level MVCC lookups. Borrowed +/// from the [`super::IndexStore`]; cheap to construct (it holds a slice). +pub struct PkLookup<'a> { + columns: &'a [Arc], +} + +impl<'a> PkLookup<'a> { + pub(super) fn new(columns: &'a [Arc]) -> Self { + Self { columns } + } + + /// The newest row position of the tuple `values` (in primary-key order) + /// visible at `max_visible_row`, or `None` if no version is visible. + /// + /// `values` must have one entry per primary-key column. + pub fn get_newest_visible( + &self, + values: &[ScalarValue], + max_visible_row: RowPosition, + ) -> Option { + match self.columns { + [] => None, + // Single-column primary key: one seek-and-stop, no intersection. + [only] => only.get_newest_visible(&values[0], max_visible_row), + // Composite: walk the leading column's visible positions newest-first + // and keep the first that every other column also holds at the same + // physical position (⇒ the whole tuple matches that row). + [leading, rest @ ..] => { + let mut positions = leading.visible_positions(&values[0], max_visible_row); + positions.reverse(); + positions.into_iter().find(|&position| { + rest.iter() + .zip(&values[1..]) + .all(|(column, value)| column.contains_position(value, position)) + }) + } + } + } + + /// Whether `position` is the newest visible row of `values` — the recency + /// check the active index-search arms apply to drop predicate-crossing + /// stale hits. + pub fn is_newest( + &self, + values: &[ScalarValue], + position: RowPosition, + max_visible_row: RowPosition, + ) -> bool { + self.get_newest_visible(values, max_visible_row) == Some(position) + } + + /// Whether `values` has any version visible at `max_visible_row` — the + /// cross-source block-list's existence query, snapshot-bounded so a + /// not-yet-visible write can't shadow an older visible copy. + pub fn contains_visible(&self, values: &[ScalarValue], max_visible_row: RowPosition) -> bool { + self.get_newest_visible(values, max_visible_row).is_some() + } + + /// Whether the primary-key index holds no rows. All columns are inserted + /// together, so the leading column's emptiness answers for the tuple. + pub fn is_empty(&self) -> bool { + self.columns.first().is_none_or(|c| c.is_empty()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow_array::{Int32Array, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + + fn btree(field_id: i32, column: &str) -> Arc { + Arc::new(BTreeMemIndex::new(field_id, column.to_string())) + } + + fn composite_batch(ids: &[i32], names: &[&str]) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + RecordBatch::try_new( + schema, + vec![ + Arc::new(Int32Array::from(ids.to_vec())), + Arc::new(StringArray::from(names.to_vec())), + ], + ) + .unwrap() + } + + #[test] + fn single_column_matches_the_btree_directly() { + let id = btree(0, "id"); + // id=1 at positions 0 and 2 (an update), id=2 at position 1. + let b = Int32Array::from(vec![1, 2]); + id.insert( + &RecordBatch::try_new( + Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])), + vec![Arc::new(b)], + ) + .unwrap(), + 0, + ) + .unwrap(); + id.insert( + &RecordBatch::try_new( + Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])), + vec![Arc::new(Int32Array::from(vec![1]))], + ) + .unwrap(), + 2, + ) + .unwrap(); + + let columns = [id]; + let pk = PkLookup::new(&columns); + let one = [ScalarValue::Int32(Some(1))]; + // Watermark above the update sees the newest position; below it, the older. + assert_eq!(pk.get_newest_visible(&one, 5), Some(2)); + assert_eq!(pk.get_newest_visible(&one, 1), Some(0)); + assert!(pk.is_newest(&one, 2, 5)); + assert!(!pk.is_newest(&one, 0, 5)); + } + + #[test] + fn composite_intersects_by_position() { + let id = btree(0, "id"); + let name = btree(1, "name"); + // Rows: (1,"a")@0, (1,"b")@1, (1,"a")@2 — an update of (1,"a"). + let b = composite_batch(&[1, 1, 1], &["a", "b", "a"]); + id.insert(&b, 0).unwrap(); + name.insert(&b, 0).unwrap(); + + let columns = [id, name]; + let pk = PkLookup::new(&columns); + let tuple_1a = [ScalarValue::Int32(Some(1)), ScalarValue::from("a")]; + let tuple_1b = [ScalarValue::Int32(Some(1)), ScalarValue::from("b")]; + + // (1,"a") newest visible row is its re-write at position 2. + assert_eq!(pk.get_newest_visible(&tuple_1a, 5), Some(2)); + // (1,"b") only exists at position 1. + assert_eq!(pk.get_newest_visible(&tuple_1b, 5), Some(1)); + // The stale (1,"a")@0 is not the newest; the re-write @2 is. + assert!(!pk.is_newest(&tuple_1a, 0, 5)); + assert!(pk.is_newest(&tuple_1a, 2, 5)); + // Watermark below the re-write: the older (1,"a")@0 is the newest visible. + assert_eq!(pk.get_newest_visible(&tuple_1a, 1), Some(0)); + assert!(pk.is_newest(&tuple_1a, 0, 1)); + // An absent tuple. + let tuple_2a = [ScalarValue::Int32(Some(2)), ScalarValue::from("a")]; + assert_eq!(pk.get_newest_visible(&tuple_2a, 5), None); + assert!(!pk.contains_visible(&tuple_2a, 5)); + } +} diff --git a/rust/lance/src/dataset/mem_wal/index/pk_position.rs b/rust/lance/src/dataset/mem_wal/index/pk_position.rs deleted file mode 100644 index 1a78e12ada1..00000000000 --- a/rust/lance/src/dataset/mem_wal/index/pk_position.rs +++ /dev/null @@ -1,228 +0,0 @@ -// SPDX-License-Identifier: Apache-2.0 -// SPDX-FileCopyrightText: Copyright The Lance Authors - -//! Maintained MVCC primary-key → newest-position index for in-memory memtables. -//! -//! Keyed on `(pk_hash, row_position)` in the same lock-free arena skiplist the -//! [`super::BTreeMemIndex`] uses, so every version of a primary key is a -//! distinct, position-ordered entry — an MVCC version chain -//! `(k,p0),(k,p1),(k,p2)…`. [`Self::get_newest_visible`] answers "the newest -//! row position of this PK that is visible at the watermark" with a single -//! seek-and-stop, the same primitive `BTreeMemIndex::get_newest_visible` -//! exposes and that point-lookup already trusts — but keyed on -//! [`compute_pk_hash`] rather than a single column, so it covers composite and -//! otherwise-unindexed primary keys uniformly. -//! -//! The active vector / FTS search arms use this to drop a stale hit whose PK -//! has a newer version that the (append-only) secondary index didn't return — -//! the predicate-crossing stale read those arms otherwise leak. Because the row -//! position is itself the MVCC version stamp, a reader filtering on its latched -//! `max_visible` watermark is unaffected by concurrent appends (which only add -//! larger positions), so no snapshot needs to be co-published with the query. - -use std::sync::Mutex; - -use arrow_array::RecordBatch; -use lance_core::{Error, Result}; - -use super::RowPosition; -use super::arena_skiplist::{SkipListReader, SkipListWriter, new_skiplist}; -use crate::dataset::mem_wal::scanner::exec::{compute_pk_hash, resolve_pk_indices}; - -/// Skiplist key: `(pk_hash, row_position)`. Sorting by hash then position means -/// a seek to `(hash, watermark)` lands on the newest version of that hash at or -/// below the watermark. The row position keeps every entry unique. -#[derive(Clone, Copy, PartialEq, Eq, PartialOrd, Ord)] -struct PkPosKey { - hash: u64, - position: RowPosition, -} - -/// Append-only, lock-free-read index from `compute_pk_hash(pk_columns)` to the -/// row positions that key was written at. Single-writer (the MemTable serializes -/// inserts behind an uncontended `Mutex`); reads take no lock. -pub struct PkPositionIndex { - reader: SkipListReader, - writer: Mutex>, - /// Primary-key column names, resolved to indices against each batch's schema. - pk_columns: Vec, -} - -impl std::fmt::Debug for PkPositionIndex { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("PkPositionIndex") - .field("pk_columns", &self.pk_columns) - .field("len", &self.len()) - .finish() - } -} - -impl PkPositionIndex { - /// Create an index over `pk_columns` (the unenforced primary key). - pub fn new(pk_columns: Vec) -> Self { - let (writer, reader) = new_skiplist::(); - Self { - reader, - writer: Mutex::new(writer), - pk_columns, - } - } - - /// Insert every row's `(pk_hash, row_offset + row_idx)`. - pub fn insert(&self, batch: &RecordBatch, row_offset: u64) -> Result<()> { - if batch.num_rows() == 0 { - return Ok(()); - } - let pk_indices = resolve_pk_indices(batch, &self.pk_columns) - .map_err(|e| Error::invalid_input(e.to_string()))?; - let mut writer = self.writer.lock().unwrap(); - for row in 0..batch.num_rows() { - let hash = compute_pk_hash(batch, &pk_indices, row); - writer.insert(PkPosKey { - hash, - position: row_offset + row as u64, - }); - } - Ok(()) - } - - /// The newest row position written for `pk_hash` that is `<= max_visible_row` - /// (inclusive), or `None` if the key has no visible version. A single - /// seek-and-stop on the skiplist (largest key `<= (pk_hash, max_visible_row)`) - /// — no range collect, no allocation. - pub fn get_newest_visible( - &self, - pk_hash: u64, - max_visible_row: RowPosition, - ) -> Option { - let target = PkPosKey { - hash: pk_hash, - position: max_visible_row, - }; - self.reader - .upper_bound_with(&target, |key| (key.hash == pk_hash).then_some(key.position)) - .flatten() - } - - /// Whether `pk_hash` has any version visible at `max_visible_row`. The - /// cross-source block-list's existence query — "does a newer generation - /// contain this PK?" — reduces to this, position-bounded so a not-yet-visible - /// write can't shadow an older visible copy. - pub fn contains_visible(&self, pk_hash: u64, max_visible_row: RowPosition) -> bool { - self.get_newest_visible(pk_hash, max_visible_row).is_some() - } - - /// All row positions written for `pk_hash` that are `<= max_visible_row`, in - /// ascending (oldest-first) order. Used by point-lookup to resolve a hash - /// collision: walk the matches newest-first and keep the first whose actual - /// primary-key value equals the query, so a colliding key never returns the - /// wrong row. Empty (and allocation-free past the seek) when the hash is - /// absent; the common no-collision lookup uses [`Self::get_newest_visible`]. - pub fn visible_positions( - &self, - pk_hash: u64, - max_visible_row: RowPosition, - ) -> Vec { - let start = PkPosKey { - hash: pk_hash, - position: 0, - }; - let mut positions = Vec::new(); - for key in self.reader.range_from(&start) { - if key.hash != pk_hash { - break; - } - if key.position <= max_visible_row { - positions.push(key.position); - } - } - positions - } - - /// Number of entries (one per inserted row, not per distinct key). - pub fn len(&self) -> usize { - self.reader.len() - } - - /// Whether the index has no entries. - pub fn is_empty(&self) -> bool { - self.len() == 0 - } -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow_array::{Int32Array, StringArray}; - use arrow_schema::{DataType, Field, Schema}; - use std::sync::Arc; - - fn schema() -> Arc { - Arc::new(Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("name", DataType::Utf8, true), - ])) - } - - fn batch(ids: &[i32], names: &[&str]) -> RecordBatch { - RecordBatch::try_new( - schema(), - vec![ - Arc::new(Int32Array::from(ids.to_vec())), - Arc::new(StringArray::from(names.to_vec())), - ], - ) - .unwrap() - } - - /// Hash a single-column `id` PK the way the index does, so a test can probe - /// `get_newest_visible` by value. - fn hash_id(id: i32) -> u64 { - let b = batch(&[id], &["x"]); - let pk_indices = resolve_pk_indices(&b, &["id".to_string()]).unwrap(); - compute_pk_hash(&b, &pk_indices, 0) - } - - #[test] - fn newest_visible_tracks_updates_under_watermark() { - let index = PkPositionIndex::new(vec!["id".to_string()]); - // id=1 at positions 0 and 3 (an update); id=2 at position 1. - index.insert(&batch(&[1, 2], &["a", "b"]), 0).unwrap(); - index.insert(&batch(&[1], &["a2"]), 3).unwrap(); - - // Watermark above the update sees the newest position. - assert_eq!(index.get_newest_visible(hash_id(1), 5), Some(3)); - assert_eq!(index.get_newest_visible(hash_id(2), 5), Some(1)); - // Watermark below the update hides it — the older position wins. - assert_eq!(index.get_newest_visible(hash_id(1), 2), Some(0)); - // Watermark below every version of a key. - assert_eq!(index.get_newest_visible(hash_id(1), 0), Some(0)); - assert_eq!(index.get_newest_visible(hash_id(2), 0), None); - // Absent key. - assert_eq!(index.get_newest_visible(hash_id(999), 5), None); - } - - #[test] - fn composite_pk_is_hashed_as_a_tuple() { - // Two-column PK (id, name): (1,"a") and (1,"b") are distinct keys. - let index = PkPositionIndex::new(vec!["id".to_string(), "name".to_string()]); - index.insert(&batch(&[1, 1], &["a", "b"]), 0).unwrap(); - - let b = batch(&[1], &["a"]); - let pk_indices = resolve_pk_indices(&b, &["id".to_string(), "name".to_string()]).unwrap(); - let hash_1a = compute_pk_hash(&b, &pk_indices, 0); - let b2 = batch(&[1], &["b"]); - let hash_1b = compute_pk_hash(&b2, &pk_indices, 0); - - assert_eq!(index.get_newest_visible(hash_1a, 10), Some(0)); - assert_eq!(index.get_newest_visible(hash_1b, 10), Some(1)); - assert_ne!(hash_1a, hash_1b); - } - - #[test] - fn empty_batch_is_a_noop() { - let index = PkPositionIndex::new(vec!["id".to_string()]); - index.insert(&batch(&[], &[]), 0).unwrap(); - assert!(index.is_empty()); - } -} diff --git a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs index ed6d5558630..86939ba2eca 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs @@ -4,12 +4,12 @@ //! Per-source block-list construction for LSM vector search. //! //! A generation's membership is a [`GenMembership`]: in-memory generations -//! (active / frozen) are **probed** against their maintained PK-position index -//! (no per-query set), while on-disk generations (flushed, base) carry a cached -//! `Arc>` of PK hashes ([`compute_pk_hash`]). Each source gets a -//! `Vec` of the newer generations (`NEWER(G)`; base: all of -//! them); the KNN drops a candidate whose PK is in any (see -//! [`super::exec::PkHashFilterExec`]). +//! (active / frozen) are **probed by value** against their maintained +//! primary-key BTrees (no per-query set), while on-disk generations (flushed, +//! base) carry a cached `Arc>` of PK hashes ([`compute_pk_hash`]). +//! Each source gets a `Vec` of the newer generations +//! (`NEWER(G)`; base: all of them); the KNN drops a candidate whose PK is in any +//! (see [`super::exec::PkHashFilterExec`]). //! //! Cross-generation only: within-gen dups share a hash and fall to the global //! dedup's `(generation, freshness)` tiebreaker. @@ -18,6 +18,7 @@ use std::collections::{HashMap, HashSet}; use std::sync::Arc; use arrow_array::RecordBatch; +use datafusion::common::ScalarValue; use futures::TryStreamExt; use lance_core::Result; @@ -31,11 +32,11 @@ use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; use crate::session::Session; /// One newer generation's PK membership, used to decide whether it shadows an -/// older source's row. In-memory generations (active / frozen) are **probed** -/// against their maintained MVCC PK-position index — no per-query set is built — -/// while on-disk generations (flushed, base) carry a materialized PK-hash set -/// (cached by path). The probe is snapshot-bounded, so a not-yet-visible write -/// can't shadow an older visible copy. +/// older source's row. In-memory generations (active / frozen) are **probed by +/// value** against their maintained primary-key BTrees — no per-query set is +/// built — while on-disk generations (flushed, base) carry a materialized +/// PK-hash set (cached by path). The probe is snapshot-bounded, so a +/// not-yet-visible write can't shadow an older visible copy. #[derive(Debug, Clone)] pub enum GenMembership { /// Probe the in-memory memtable's index, bounded to its visible prefix. @@ -45,13 +46,16 @@ pub enum GenMembership { max_visible_row: Option, }, /// A materialized PK-hash set (flushed / base, or an in-memory memtable that - /// has no PK-position index). + /// has no primary-key index). Set(Arc>), } impl GenMembership { - /// Whether this generation contains `pk_hash` (visibly, for the index case). - pub fn contains(&self, pk_hash: u64) -> bool { + /// Whether this generation visibly contains the primary key. The on-disk + /// `Set` case matches on `pk_hash`; the in-memory `Index` case probes by + /// `pk_values` (collision-free). The caller supplies both for a candidate + /// row — see [`Self::needs_values`]. + pub fn contains(&self, pk_hash: u64, pk_values: &[ScalarValue]) -> bool { match self { Self::Index { index_store, @@ -61,13 +65,20 @@ impl GenMembership { return false; }; index_store - .pk_position_index() - .is_some_and(|idx| idx.contains_visible(pk_hash, *max)) + .pk_index() + .is_some_and(|idx| idx.contains_visible(pk_values, *max)) } Self::Set(set) => set.contains(&pk_hash), } } + /// Whether [`Self::contains`] needs the primary-key values (the `Index` + /// case) rather than just the hash. Lets the filter skip per-row value + /// extraction when every membership is an on-disk `Set`. + pub fn needs_values(&self) -> bool { + matches!(self, Self::Index { .. }) + } + /// Whether this generation has no (visible) membership — used to skip adding /// an empty blocked set. Approximate for the index case (it ignores the /// watermark when counting), which only ever leaves a harmless no-op entry. @@ -77,10 +88,7 @@ impl GenMembership { index_store, max_visible_row, } => { - max_visible_row.is_none() - || index_store - .pk_position_index() - .is_none_or(|idx| idx.is_empty()) + max_visible_row.is_none() || index_store.pk_index().is_none_or(|idx| idx.is_empty()) } Self::Set(set) => set.is_empty(), } @@ -198,7 +206,7 @@ pub async fn fresh_tier_block_list( /// Cross-source membership of an in-memory (active / frozen) memtable. /// -/// Prefers a snapshot-bounded **probe** of the maintained PK-position index (no +/// Prefers a snapshot-bounded **probe** of the maintained primary-key index (no /// per-query set built), falling back to a one-time `BatchStore` scan only when /// the memtable has no such index (e.g. a table without a primary key) — which /// the production vector-search path never hits, since that index is always @@ -208,7 +216,7 @@ fn in_memory_membership( index_store: &Arc, pk_columns: &[String], ) -> Result { - if index_store.pk_position_index().is_some() { + if index_store.pk_index().is_some() { let max_visible_row = batch_store.max_visible_row(index_store.max_visible_batch_position()); Ok(GenMembership::Index { index_store: index_store.clone(), @@ -333,10 +341,11 @@ mod tests { compute_pk_hash(&batch, &pk_indices, 0) } - /// Whether `id`'s PK hash is blocked by any of a source's newer-gen - /// memberships. + /// Whether `id`'s PK is blocked by any of a source's newer-gen memberships + /// (supplying both the hash and the value, as the filter does). fn blocks(memberships: &[GenMembership], id: i32) -> bool { - memberships.iter().any(|m| m.contains(hash_id(id))) + let values = [ScalarValue::Int32(Some(id))]; + memberships.iter().any(|m| m.contains(hash_id(id), &values)) } #[test] @@ -392,7 +401,7 @@ mod tests { // One membership per generation; together they cover pk=1,2,3 (not 4). assert_eq!(memberships.len(), 2); - let fresh_blocks = |id: i32| memberships.iter().any(|m| m.contains(hash_id(id))); + let fresh_blocks = |id: i32| blocks(&memberships, id); for id in [1, 2, 3] { assert!(fresh_blocks(id)); } @@ -550,7 +559,7 @@ mod tests { } #[tokio::test] - async fn in_memory_membership_reads_from_pk_position_index() { + async fn in_memory_membership_reads_from_pk_index() { // When the memtable has a maintained PK-position index, the block-list // sources its membership from that index (no BatchStore re-scan) and // still suppresses an older generation's stale copy. @@ -569,7 +578,7 @@ mod tests { // so the snapshot-bounded probe sees both rows. let active_store = BatchStore::with_capacity(8); let mut active_index = IndexStore::new(); - active_index.enable_pk_position_index(vec!["id".to_string()]); + active_index.enable_pk_index(&[("id".to_string(), 0)]); let b1 = id_batch(&[1]); let (bp1, off1, _) = active_store.append(b1.clone()).unwrap(); active_index @@ -637,7 +646,7 @@ mod tests { // index yet not visible) — the concurrent-write race. let g2_store = BatchStore::with_capacity(8); let mut g2_index = IndexStore::new(); - g2_index.enable_pk_position_index(vec!["id".to_string()]); + g2_index.enable_pk_index(&[("id".to_string(), 0)]); let b0 = id_batch(&[99]); let (bp0, off0, _) = g2_store.append(b0.clone()).unwrap(); g2_index diff --git a/rust/lance/src/dataset/mem_wal/scanner/builder.rs b/rust/lance/src/dataset/mem_wal/scanner/builder.rs index 59149ef93ba..1f6df3d1b32 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/builder.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/builder.rs @@ -466,12 +466,26 @@ impl LsmScanner { .await?; let pk_indices = super::exec::resolve_pk_indices(pks, &self.pk_columns) .map_err(|e| Error::invalid_input(e.to_string()))?; - Ok((0..pks.num_rows()) - .map(|row| { - let hash = super::exec::compute_pk_hash(pks, &pk_indices, row); - memberships.iter().any(|m| m.contains(hash)) - }) - .collect()) + // On-disk generations probe by hash; in-memory ones probe their + // primary-key BTrees by value. Extract values only when needed. + let needs_values = memberships + .iter() + .any(super::block_list::GenMembership::needs_values); + let mut contained = Vec::with_capacity(pks.num_rows()); + for row in 0..pks.num_rows() { + let hash = super::exec::compute_pk_hash(pks, &pk_indices, row); + let values: Vec = if needs_values { + pk_indices + .iter() + .map(|&col| ScalarValue::try_from_array(pks.column(col), row)) + .collect::>() + .map_err(|e| Error::invalid_input(e.to_string()))? + } else { + Vec::new() + }; + contained.push(memberships.iter().any(|m| m.contains(hash, &values))); + } + Ok(contained) } /// Build the data source collector. diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs index a9ad14856c4..b59b083da02 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs @@ -10,13 +10,13 @@ //! stale version against — and it leaks. //! //! This node closes that hole with a predicate-independent recency check: for -//! each hit it asks the memtable's maintained MVCC PK-position index -//! ([`crate::dataset::mem_wal::index::PkPositionIndex`]) for the newest position -//! of that hit's primary key visible at the query's `max_visible` watermark, and -//! keeps the hit **iff that equals the hit's own row position**. A stale hit -//! (some newer version exists) is dropped even when that newer version never -//! appears in the result. This is exactly the seek point-lookup already does; -//! the index search arms simply didn't do it. +//! each hit it asks the memtable's maintained primary-key index +//! ([`crate::dataset::mem_wal::index::PkLookup`]) whether the hit's own row +//! position is the newest version of its primary key visible at the query's +//! `max_visible` watermark, and keeps the hit **iff so**. A stale hit (some +//! newer version exists) is dropped even when that newer version never appears +//! in the result. This is exactly the seek point-lookup already does; the index +//! search arms simply didn't do it. use std::any::Any; use std::fmt; @@ -27,6 +27,7 @@ use std::task::{Context, Poll}; use arrow::compute::filter_record_batch; use arrow_array::{Array, BooleanArray, RecordBatch, UInt64Array}; use arrow_schema::SchemaRef; +use datafusion::common::ScalarValue; use datafusion::error::{DataFusionError, Result as DFResult}; use datafusion::execution::TaskContext; use datafusion::physical_expr::EquivalenceProperties; @@ -36,7 +37,7 @@ use datafusion::physical_plan::{ }; use futures::{Stream, StreamExt}; -use super::pk::{compute_pk_hash, resolve_pk_indices}; +use super::pk::resolve_pk_indices; use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; /// Keeps only the index hits that are the newest visible version of their PK. @@ -47,7 +48,7 @@ pub struct NewestPkFilterExec { input: Arc, pk_columns: Vec, row_id_column: String, - /// Holds the maintained `PkPositionIndex` queried per hit. + /// Holds the maintained primary-key index (`PkLookup`) queried per hit. index_store: Arc, /// Resolves the `max_visible` row watermark from the visible batch prefix. batch_store: Arc, @@ -194,9 +195,9 @@ struct NewestPkFilterStream { impl NewestPkFilterStream { fn filter_batch(&self, batch: RecordBatch) -> DFResult { - // No PK-position index (memtable without a primary key), no visible + // No primary-key index (memtable without a primary key), no visible // rows, or an empty batch: nothing to dedup against, so pass it through. - let Some(pk_index) = self.index_store.pk_position_index() else { + let Some(pk_index) = self.index_store.pk_index() else { return Ok(batch); }; let Some(max_visible_row) = self.max_visible_row else { @@ -224,20 +225,23 @@ impl NewestPkFilterStream { )) })?; - let keep: BooleanArray = (0..batch.num_rows()) - .map(|row| { - // A null row position can't be ordered; keep it rather than - // guess (callers always project a real position here). - if row_ids.is_null(row) { - return true; - } - let position = row_ids.value(row); - let hash = compute_pk_hash(&batch, &pk_indices, row); - // Keep iff this hit is the newest visible version of its PK. - pk_index.get_newest_visible(hash, max_visible_row) == Some(position) - }) - .collect(); - filter_record_batch(&batch, &keep) + let mut keep = Vec::with_capacity(batch.num_rows()); + for row in 0..batch.num_rows() { + // A null row position can't be ordered; keep it rather than guess + // (callers always project a real position here). + if row_ids.is_null(row) { + keep.push(true); + continue; + } + let position = row_ids.value(row); + let values: Vec = pk_indices + .iter() + .map(|&col| ScalarValue::try_from_array(batch.column(col), row)) + .collect::>()?; + // Keep iff this hit is the newest visible version of its PK. + keep.push(pk_index.is_newest(&values, position, max_visible_row)); + } + filter_record_batch(&batch, &BooleanArray::from(keep)) .map_err(|e| DataFusionError::ArrowError(Box::new(e), None)) } } @@ -293,12 +297,12 @@ mod tests { .unwrap() } - /// Build an active memtable whose PK-position index + BatchStore hold one - /// row per `id` in `appended` (positions 0..n), all committed. + /// Build an active memtable whose PK index + BatchStore hold one row per + /// `id` in `appended` (positions 0..n), all committed. fn active(appended: &[i32]) -> (Arc, Arc) { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index = IndexStore::new(); - index.enable_pk_position_index(vec!["id".to_string()]); + index.enable_pk_index(&[("id".to_string(), 0)]); for &id in appended { let b = id_batch(id); let (bp, off, _) = batch_store.append(b.clone()).unwrap(); @@ -373,12 +377,12 @@ mod tests { } #[tokio::test] - async fn passes_through_when_no_pk_position_index() { - // A memtable without a PK-position index can't be deduped here, so the + async fn passes_through_when_no_pk_index() { + // A memtable without a primary-key index can't be deduped here, so the // filter is a pass-through rather than dropping everything. let batch_store = Arc::new(BatchStore::with_capacity(16)); batch_store.append(id_batch(1)).unwrap(); - let index = Arc::new(IndexStore::new()); // no enable_pk_position_index + let index = Arc::new(IndexStore::new()); // no enable_pk_index let rows = run(index, batch_store, 0, hits(&[(1, 0), (1, 9)])).await; assert_eq!(rows, vec![(1, 0), (1, 9)]); } diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs index 6366b654407..d32b7993911 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs @@ -22,6 +22,7 @@ use std::task::{Context, Poll}; use arrow::compute::filter_record_batch; use arrow_array::{BooleanArray, RecordBatch}; use arrow_schema::SchemaRef; +use datafusion::common::ScalarValue; use datafusion::error::{DataFusionError, Result as DFResult}; use datafusion::execution::TaskContext; use datafusion::physical_expr::EquivalenceProperties; @@ -164,16 +165,29 @@ impl PkHashFilterStream { return Ok(batch); } let pk_indices = resolve_pk_indices(&batch, &self.pk_columns)?; - let keep: BooleanArray = (0..batch.num_rows()) - .map(|row| { - let hash = compute_pk_hash(&batch, &pk_indices, row); + // On-disk generations probe by hash; in-memory ones probe their + // primary-key BTrees by value. Extract per-row values only when some + // membership needs them. + let needs_values = self.blocked.iter().any(GenMembership::needs_values); + let mut keep = Vec::with_capacity(batch.num_rows()); + for row in 0..batch.num_rows() { + let hash = compute_pk_hash(&batch, &pk_indices, row); + let values: Vec = if needs_values { + pk_indices + .iter() + .map(|&col| ScalarValue::try_from_array(batch.column(col), row)) + .collect::>()? + } else { + Vec::new() + }; + keep.push( !self .blocked .iter() - .any(|membership| membership.contains(hash)) - }) - .collect(); - filter_record_batch(&batch, &keep) + .any(|membership| membership.contains(hash, &values)), + ); + } + filter_record_batch(&batch, &BooleanArray::from(keep)) .map_err(|e| DataFusionError::ArrowError(Box::new(e), None)) } } diff --git a/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs b/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs index 490a34ee366..74b3358d167 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/fts_search.rs @@ -420,7 +420,7 @@ mod tests { // Active memtable with its own FTS index, containing a matching row. let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut indexes = IndexStore::new(); - indexes.enable_pk_position_index(vec!["id".to_string()]); + indexes.enable_pk_index(&[("id".to_string(), 0)]); indexes.add_fts("text_fts".to_string(), 1, "text".to_string()); let active_batch = make_batch( &schema, @@ -500,7 +500,7 @@ mod tests { let schema = fts_schema(); let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut indexes = IndexStore::new(); - indexes.enable_pk_position_index(vec!["id".to_string()]); + indexes.enable_pk_index(&[("id".to_string(), 0)]); // text column has field_id 1 in fts_schema() indexes.add_fts("text_fts".to_string(), 1, "text".to_string()); let batch = make_batch( @@ -597,7 +597,7 @@ mod tests { let schema = fts_schema(); let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut indexes = IndexStore::new(); - indexes.enable_pk_position_index(vec!["id".to_string()]); + indexes.enable_pk_index(&[("id".to_string(), 0)]); indexes.add_fts("text_fts".to_string(), 1, "text".to_string()); // Insert pk=1 ("alpha lance") and an unrelated live pk=2 ("alpha foo"). diff --git a/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs b/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs index 6fc71abd7e7..1107cac1bfc 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs @@ -669,47 +669,20 @@ fn probe_position( } let max_visible_row = visible_end - 1; - // Prefer a value-keyed scalar BTree on the PK column when one exists: it is - // collision-free, so a single seek-and-stop yields the answer with no value - // re-check. - if let Some(btree) = index_store.get_btree_by_column(pk_column) { - let Some(pos) = btree.get_newest_visible(pk_value, max_visible_row) else { - return Ok(ProbePos::Miss); - }; - let (batch_idx, row) = resolve_position(batch_store, last_visible_idx, pos)?; - return Ok(ProbePos::Found { batch_idx, row }); - } - - // Otherwise probe the maintained PK-position index, which is hash-keyed and - // present whenever the table has a primary key (so it covers PK columns - // without their own scalar BTree). Hashing can collide, so verify the - // candidate's actual value: walk this hash's visible positions newest-first - // and return the first whose value equals the query — a colliding key never - // returns the wrong row. The common (no-collision) case checks one position. - let Some(pk_index) = index_store.pk_position_index() else { + // A single-column primary key always has a value-keyed BTree on it (a + // reused user index or the auto-created `__pk__*` one — see + // `IndexStore::enable_pk_index`). It is collision-free, so one seek-and-stop + // yields the answer with no value re-check. Absent only when the table has + // no primary-key index at all, in which case the caller falls back to the + // plan path. + let Some(btree) = index_store.get_btree_by_column(pk_column) else { return Ok(ProbePos::NoIndex); }; - let hash = compute_pk_hash_from_scalars(std::slice::from_ref(pk_value)); - for pos in pk_index - .visible_positions(hash, max_visible_row) - .into_iter() - .rev() - { - let (batch_idx, row) = resolve_position(batch_store, last_visible_idx, pos)?; - let stored = batch_store - .get(batch_idx) - .ok_or_else(|| lance_core::Error::internal("point-lookup: resolved batch missing"))?; - let col_idx = stored.data.schema_ref().index_of(pk_column).map_err(|_| { - lance_core::Error::internal(format!( - "point-lookup: PK column '{pk_column}' not found in memtable batch" - )) - })?; - let actual = ScalarValue::try_from_array(stored.data.column(col_idx), row)?; - if &actual == pk_value { - return Ok(ProbePos::Found { batch_idx, row }); - } - } - Ok(ProbePos::Miss) + let Some(pos) = btree.get_newest_visible(pk_value, max_visible_row) else { + return Ok(ProbePos::Miss); + }; + let (batch_idx, row) = resolve_position(batch_store, last_visible_idx, pos)?; + Ok(ProbePos::Found { batch_idx, row }) } /// Map a global row `position` to its `(batch_idx, row_in_batch)` by binary @@ -1216,11 +1189,11 @@ mod tests { } #[tokio::test] - async fn test_point_lookup_probes_pk_position_index_without_btree() { - // No scalar BTree on the PK column, only the maintained PK-position - // index (the production default). The fast probe must resolve the newest - // visible version through that index — verifying the value to guard hash - // collisions — rather than falling back to the plan path. + async fn test_point_lookup_probes_auto_created_pk_btree() { + // No user `add_btree` on the PK column — only `enable_pk_index`, which + // auto-creates a BTree on the primary key (the production default). The + // fast probe must resolve the newest visible version through that + // collision-free BTree rather than falling back to the plan path. use crate::dataset::mem_wal::scanner::collector::{InMemoryMemTableRef, InMemoryMemTables}; use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; @@ -1230,8 +1203,8 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - // No `add_btree` — only the PK-position index. - index_store.enable_pk_position_index(vec!["id".to_string()]); + // No `add_btree` — `enable_pk_index` auto-creates the PK BTree. + index_store.enable_pk_index(&[("id".to_string(), 0)]); // pk=1 written twice (the newer second), plus an unrelated pk=2. let b_old = create_test_batch(&schema, &[1], "old"); diff --git a/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs b/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs index a27b82d9deb..4036ef31d17 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs @@ -660,7 +660,7 @@ mod tests { // Active memtable with HNSW index over the "vector" column. let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id".to_string()]); + index_store.enable_pk_index(&[("id".to_string(), 0)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -779,7 +779,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id".to_string()]); + index_store.enable_pk_index(&[("id".to_string(), 0)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -859,7 +859,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id".to_string()]); + index_store.enable_pk_index(&[("id".to_string(), 0)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -973,7 +973,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id".to_string()]); + index_store.enable_pk_index(&[("id".to_string(), 0)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1113,7 +1113,7 @@ mod tests { // "right" vector close to the query, plus an unrelated pk=2. let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id".to_string()]); + index_store.enable_pk_index(&[("id".to_string(), 0)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1233,7 +1233,7 @@ mod tests { // Active memtable: id=3 with HNSW index. let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id".to_string()]); + index_store.enable_pk_index(&[("id".to_string(), 0)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1474,7 +1474,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id".to_string()]); + index_store.enable_pk_index(&[("id".to_string(), 0)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1603,7 +1603,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id".to_string()]); + index_store.enable_pk_index(&[("id".to_string(), 0)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1722,7 +1722,7 @@ mod tests { // active arm surfaces pk=2 and drops fresh pk=1. let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id".to_string()]); + index_store.enable_pk_index(&[("id".to_string(), 0)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -1919,7 +1919,7 @@ mod tests { // Active (gen 1): pk 1,2,3 re-inserted with a far vector (the fresh value). let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id".to_string()]); + index_store.enable_pk_index(&[("id".to_string(), 0)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -2124,7 +2124,7 @@ mod tests { // Active: (1,1) re-inserted far (fresh) + an unrelated nearby (2,2). let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id1".to_string(), "id2".to_string()]); + index_store.enable_pk_index(&[("id1".to_string(), 0), ("id2".to_string(), 1)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, @@ -2229,7 +2229,7 @@ mod tests { let batch_store = Arc::new(BatchStore::with_capacity(16)); let mut index_store = IndexStore::new(); - index_store.enable_pk_position_index(vec!["id".to_string()]); + index_store.enable_pk_index(&[("id".to_string(), 0)]); index_store.add_hnsw( "vector_hnsw".to_string(), 1, diff --git a/rust/lance/src/dataset/mem_wal/write.rs b/rust/lance/src/dataset/mem_wal/write.rs index d2d6eb27ae8..b931d49bafa 100644 --- a/rust/lance/src/dataset/mem_wal/write.rs +++ b/rust/lance/src/dataset/mem_wal/write.rs @@ -814,6 +814,16 @@ async fn replay_memtable_from_wal( Ok(position) } +/// Pair each primary-key column name with its field id (both derived from the +/// schema's primary key, in the same order) for [`IndexStore::enable_pk_index`]. +fn pk_index_columns(pk_columns: &[String], pk_field_ids: &[i32]) -> Vec<(String, i32)> { + pk_columns + .iter() + .cloned() + .zip(pk_field_ids.iter().copied()) + .collect() +} + /// Shared state for writer operations. struct SharedWriterState { state: Arc>, @@ -886,7 +896,7 @@ impl SharedWriterState { self.max_memtable_rows, self.max_memtable_batches, )?; - indexes.enable_pk_position_index(self.pk_columns.clone()); + indexes.enable_pk_index(&pk_index_columns(&self.pk_columns, &self.pk_field_ids)); new_memtable.set_indexes_arc(Arc::new(indexes)); } @@ -1275,7 +1285,7 @@ impl ShardWriter { config.max_memtable_rows, config.max_memtable_batches, )?; - indexes.enable_pk_position_index(pk_columns.clone()); + indexes.enable_pk_index(&pk_index_columns(&pk_columns, &pk_field_ids)); memtable.set_indexes_arc(Arc::new(indexes)); } From d48bebdd41354b66c658a68fe407bb9adc03e955 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Wed, 3 Jun 2026 13:46:54 -0500 Subject: [PATCH 07/12] refactor(mem_wal): fold PkLookup into IndexStore; tighten comments PkLookup was a borrow-only view over the pk_btrees slice with three trivial methods; move them onto IndexStore (which owns the BTrees) as pk_newest_visible / pk_is_newest / pk_contains_visible + has_pk_index / pk_is_empty, and delete index/pk_lookup.rs. No wrapper type, no lifetime view. Also trims the verbose doc/comments added in the previous commit. cargo test -p lance --lib mem_wal: 385 passed; fmt + clippy clean. Co-Authored-By: Claude Opus 4.8 --- rust/lance/src/dataset/mem_wal/index.rs | 175 +++++++++++++--- rust/lance/src/dataset/mem_wal/index/btree.rs | 18 +- .../src/dataset/mem_wal/index/pk_lookup.rs | 186 ------------------ .../src/dataset/mem_wal/scanner/block_list.rs | 18 +- .../mem_wal/scanner/exec/newest_pk_filter.rs | 18 +- .../dataset/mem_wal/scanner/point_lookup.rs | 10 +- 6 files changed, 172 insertions(+), 253 deletions(-) delete mode 100644 rust/lance/src/dataset/mem_wal/index/pk_lookup.rs diff --git a/rust/lance/src/dataset/mem_wal/index.rs b/rust/lance/src/dataset/mem_wal/index.rs index 4563f33e00c..0d540bfdf48 100644 --- a/rust/lance/src/dataset/mem_wal/index.rs +++ b/rust/lance/src/dataset/mem_wal/index.rs @@ -18,12 +18,13 @@ mod arena_skiplist; mod btree; mod fts; mod hnsw; -mod pk_lookup; use std::collections::HashMap; use std::sync::Arc; use std::sync::atomic::{AtomicUsize, Ordering}; +use datafusion::common::ScalarValue; + use super::memtable::batch_store::StoredBatch; use arrow_array::RecordBatch; use lance_core::datatypes::Schema as LanceSchema; @@ -46,7 +47,6 @@ pub type RowPosition = u64; pub use btree::{BTreeIndexConfig, BTreeMemIndex}; pub use fts::{FtsIndexConfig, FtsMemIndex, FtsQueryExpr, SearchOptions}; pub use hnsw::{HnswIndexConfig, HnswMemIndex}; -pub use pk_lookup::PkLookup; // ============================================================================ // Index Store @@ -205,12 +205,10 @@ pub struct IndexStore { hnsw_indexes: HashMap, /// FTS indexes keyed by index name. fts_indexes: HashMap, - /// The primary-key BTrees, one per PK column in primary-key order. Each - /// aliases an entry in `btree_indexes` (a reused user index, or an - /// auto-created `__pk__*` one), so it is maintained by the same insert loop. - /// Empty when the memtable has no primary key. Read as a [`PkLookup`] by the - /// active vector / FTS arms and the cross-source block-list to find the - /// newest visible version of a key (see [`Self::enable_pk_index`]). + /// The primary-key BTrees, one per PK column in order. Each aliases a + /// `btree_indexes` entry (reused or auto-created), so the insert loop + /// maintains it. Empty without a primary key; queried via + /// [`Self::pk_newest_visible`] (see [`Self::enable_pk_index`]). pk_btrees: Vec>, /// Maximum batch position that is durable in the WAL and therefore /// visible to scanners. Advanced unconditionally after a WAL append @@ -382,19 +380,15 @@ impl IndexStore { .insert(name, FtsMemIndex::with_params(field_id, column, params)); } - /// Maintain a BTree index on each primary-key column so the memtable can - /// answer "newest visible version of this key" (see [`Self::pk_index`]). - /// - /// For each `(column, field_id)` an existing BTree on that field is reused; - /// otherwise an auto-created one is registered under a synthetic `__pk__*` - /// name so the normal insert loop maintains it. Call this once at - /// construction, after [`Self::from_configs`] (so user indexes are present - /// to reuse) and before any inserts. A no-op when `pk_columns` is empty (a - /// memtable without a primary key has nothing to dedup). + /// Maintain a BTree on each primary-key column so the memtable can answer + /// "newest visible version of this key" (see [`Self::pk_index`]). /// - /// The auto-created BTrees are in-memory only — flush rebuilds indexes from - /// the user's index configs, not from this map — so a primary key without a - /// user-defined scalar index does not gain an on-disk index. + /// Reuses an existing BTree on the field, else auto-creates one under a + /// `__pk__*` name so the normal insert loop maintains it. Call once at + /// construction, after [`Self::from_configs`] and before any inserts; a + /// no-op when `pk_columns` is empty. The auto-created BTrees are in-memory + /// only — flush rebuilds from the user's index configs — so a primary key + /// without a user scalar index gains no on-disk index. pub fn enable_pk_index(&mut self, pk_columns: &[(String, i32)]) { for (column, field_id) in pk_columns { let btree = match self @@ -414,10 +408,67 @@ impl IndexStore { } } - /// A tuple-level view over the primary-key BTrees, if the memtable has a - /// primary key (see [`Self::enable_pk_index`]). - pub fn pk_index(&self) -> Option> { - (!self.pk_btrees.is_empty()).then(|| PkLookup::new(&self.pk_btrees)) + /// Whether the memtable has a primary-key index (a BTree per PK column). + pub fn has_pk_index(&self) -> bool { + !self.pk_btrees.is_empty() + } + + /// The newest row position of the primary-key tuple `values` (in PK order) + /// visible at `max_visible_row`, or `None`. + /// + /// Single-column is one seek. A composite key walks the leading column's + /// visible positions newest-first and returns the first that every other + /// column also holds at the same physical position — so the whole tuple + /// matches that row. Collision-free, since `position` is the row identity. + pub fn pk_newest_visible( + &self, + values: &[ScalarValue], + max_visible_row: RowPosition, + ) -> Option { + match self.pk_btrees.as_slice() { + [] => None, + [only] => only.get_newest_visible(&values[0], max_visible_row), + [leading, rest @ ..] => { + let mut positions = leading.visible_positions(&values[0], max_visible_row); + positions.reverse(); + positions.into_iter().find(|&position| { + rest.iter() + .zip(&values[1..]) + .all(|(column, value)| column.contains_position(value, position)) + }) + } + } + } + + /// Whether `position` is the newest visible row of `values` — the recency + /// check the active index-search arms apply to drop predicate-crossing + /// stale hits. Callers gate on [`Self::has_pk_index`] first, since this is + /// `false` (drop) when the memtable has no primary-key index. + pub fn pk_is_newest( + &self, + values: &[ScalarValue], + position: RowPosition, + max_visible_row: RowPosition, + ) -> bool { + self.pk_newest_visible(values, max_visible_row) == Some(position) + } + + /// Whether `values` has any version visible at `max_visible_row` — the + /// cross-source block-list's existence query, snapshot-bounded so a + /// not-yet-visible write can't shadow an older visible copy. + pub fn pk_contains_visible( + &self, + values: &[ScalarValue], + max_visible_row: RowPosition, + ) -> bool { + self.pk_newest_visible(values, max_visible_row).is_some() + } + + /// Whether the primary-key index holds no rows (or doesn't exist). All + /// columns are inserted together, so the leading column answers for the + /// tuple. + pub fn pk_is_empty(&self) -> bool { + self.pk_btrees.first().is_none_or(|c| c.is_empty()) } /// Insert a batch into all indexes. @@ -442,8 +493,7 @@ impl IndexStore { for index in self.fts_indexes.values() { index.insert(batch, row_offset)?; } - // The primary-key BTrees are aliases of `btree_indexes` entries, so the - // loop above already maintained them. + // PK BTrees alias `btree_indexes` entries — already maintained above. // Update global watermark after all indexes have been updated if let Some(bp) = batch_position { @@ -500,8 +550,7 @@ impl IndexStore { } } - // The primary-key BTrees are aliases of `btree_indexes` entries, so the - // loop above already maintained them. + // PK BTrees alias `btree_indexes` entries — already maintained above. // Update global watermark to the max batch position let max_bp = batches.iter().map(|b| b.batch_position).max().unwrap(); @@ -615,9 +664,8 @@ impl IndexStore { .map(|(name, _idx_type, duration)| (name.to_string(), duration)) .collect(); - // The primary-key BTrees are aliases of `btree_indexes` entries, so - // their per-index threads above already maintained them (and joined - // before the visibility watermark advances below). + // PK BTrees alias `btree_indexes` entries — their threads above + // already maintained them (and joined before the watermark advances). // Update global watermark to the max batch position let max_bp = batches.iter().map(|b| b.batch_position).max().unwrap(); @@ -763,6 +811,71 @@ mod tests { .unwrap() } + /// Single-column `id` batch for primary-key lookup tests. + fn id_batch(ids: &[i32]) -> RecordBatch { + RecordBatch::try_new( + Arc::new(ArrowSchema::new(vec![Field::new( + "id", + DataType::Int32, + false, + )])), + vec![Arc::new(Int32Array::from(ids.to_vec()))], + ) + .unwrap() + } + + #[test] + fn pk_newest_visible_single_column() { + let mut store = IndexStore::new(); + store.enable_pk_index(&[("id".to_string(), 0)]); + // id=1 at positions 0 and 2 (an update), id=2 at position 1. + store.insert(&id_batch(&[1, 2]), 0).unwrap(); + store.insert(&id_batch(&[1]), 2).unwrap(); + + let one = [ScalarValue::Int32(Some(1))]; + // Watermark above the update sees the newest position; below it, the older. + assert_eq!(store.pk_newest_visible(&one, 5), Some(2)); + assert_eq!(store.pk_newest_visible(&one, 1), Some(0)); + assert!(store.pk_is_newest(&one, 2, 5)); + assert!(!store.pk_is_newest(&one, 0, 5)); + // Absent key. + assert!(!store.pk_contains_visible(&[ScalarValue::Int32(Some(9))], 5)); + } + + #[test] + fn pk_newest_visible_composite_intersects_by_position() { + let mut store = IndexStore::new(); + store.enable_pk_index(&[("id".to_string(), 0), ("name".to_string(), 1)]); + // Rows: (1,"a")@0, (1,"b")@1, (1,"a")@2 — an update of (1,"a"). + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + let batch = RecordBatch::try_new( + schema, + vec![ + Arc::new(Int32Array::from(vec![1, 1, 1])), + Arc::new(StringArray::from(vec!["a", "b", "a"])), + ], + ) + .unwrap(); + store.insert(&batch, 0).unwrap(); + + let tuple_1a = [ScalarValue::Int32(Some(1)), ScalarValue::from("a")]; + let tuple_1b = [ScalarValue::Int32(Some(1)), ScalarValue::from("b")]; + // (1,"a")'s newest visible row is its re-write at position 2. + assert_eq!(store.pk_newest_visible(&tuple_1a, 5), Some(2)); + assert!(store.pk_is_newest(&tuple_1a, 2, 5)); + assert!(!store.pk_is_newest(&tuple_1a, 0, 5)); + // (1,"b") only exists at position 1. + assert_eq!(store.pk_newest_visible(&tuple_1b, 5), Some(1)); + // Watermark below the re-write: the older (1,"a")@0 is the newest visible. + assert_eq!(store.pk_newest_visible(&tuple_1a, 1), Some(0)); + // An absent tuple. + let tuple_2a = [ScalarValue::Int32(Some(2)), ScalarValue::from("a")]; + assert!(!store.pk_contains_visible(&tuple_2a, 5)); + } + #[test] fn test_index_registry() { let schema = create_test_schema(); diff --git a/rust/lance/src/dataset/mem_wal/index/btree.rs b/rust/lance/src/dataset/mem_wal/index/btree.rs index 30145db7293..e0123110f43 100644 --- a/rust/lance/src/dataset/mem_wal/index/btree.rs +++ b/rust/lance/src/dataset/mem_wal/index/btree.rs @@ -416,8 +416,7 @@ impl FixedIntBackend { }; let start = FixedKey { enc, position: 0 }; let mut positions = Vec::new(); - // Keys are ordered by (enc, position); for a fixed enc, positions are - // ascending, so the first position past the watermark ends the walk. + // For a fixed enc, positions ascend — stop at the first past the watermark. for key in self.reader.range_from(&start) { if key.enc != enc || key.position > max { break; @@ -976,11 +975,8 @@ impl BTreeMemIndex { } /// All row positions for `value` that are `<= max_visible_row`, ascending. - /// - /// The composite-primary-key driver: a tuple's newest visible row is found - /// by walking the leading column's visible positions newest-first and - /// keeping the first that every other column also holds at the same physical - /// position (see [`super::PkLookup`]). + /// The composite-primary-key intersection driver (see + /// [`super::IndexStore::pk_newest_visible`]). pub fn visible_positions( &self, value: &ScalarValue, @@ -992,10 +988,10 @@ impl BTreeMemIndex { .unwrap_or_default() } - /// Whether the exact `(value, position)` entry exists. One seek-and-stop: - /// the newest entry for `value` at or below `position` is `position` itself - /// iff that entry is present. Used to test a candidate row position against - /// the other primary-key columns when composing a composite key. + /// Whether the exact `(value, position)` entry exists — one seek (the newest + /// entry for `value` at or below `position` equals `position` iff present). + /// Probes a candidate position against the other PK columns (see + /// [`super::IndexStore::pk_newest_visible`]). pub fn contains_position(&self, value: &ScalarValue, position: RowPosition) -> bool { self.get_newest_visible(value, position) == Some(position) } diff --git a/rust/lance/src/dataset/mem_wal/index/pk_lookup.rs b/rust/lance/src/dataset/mem_wal/index/pk_lookup.rs deleted file mode 100644 index f04e28d89c9..00000000000 --- a/rust/lance/src/dataset/mem_wal/index/pk_lookup.rs +++ /dev/null @@ -1,186 +0,0 @@ -// SPDX-License-Identifier: Apache-2.0 -// SPDX-FileCopyrightText: Copyright The Lance Authors - -//! Tuple-level MVCC primary-key lookups over the per-column BTree indexes. -//! -//! The memtable maintains a [`super::BTreeMemIndex`] on every primary-key -//! column (reusing a user's scalar index when one exists, otherwise an -//! auto-created one — see [`super::IndexStore::enable_pk_index`]). Each BTree is -//! value-keyed as `(value, row_position)`, so for one column -//! [`super::BTreeMemIndex::get_newest_visible`] answers "the newest row position -//! for this value visible at the watermark" with a single seek. -//! -//! [`PkLookup`] composes those per-column answers into a single primary-key -//! question. Because `row_position` is the physical row identity, the rows -//! matching a tuple `(v0 … vn)` are exactly the intersection of each column's -//! position set, so the newest visible row of the tuple is the largest position -//! present in *every* column at or below the watermark. This is collision-free -//! (no hashing) and covers single and composite primary keys uniformly. -//! -//! Cost: a single-column primary key is one seek. A composite key walks the -//! leading column's positions for `v0` and point-probes the rest, so its cost -//! tracks the leading column's cardinality for that value — put the most -//! selective column first in a composite primary key. - -use std::sync::Arc; - -use datafusion::common::ScalarValue; - -use super::RowPosition; -use super::btree::BTreeMemIndex; - -/// A read-only view over the primary-key BTree indexes (one per PK column, in -/// primary-key order), composing them into tuple-level MVCC lookups. Borrowed -/// from the [`super::IndexStore`]; cheap to construct (it holds a slice). -pub struct PkLookup<'a> { - columns: &'a [Arc], -} - -impl<'a> PkLookup<'a> { - pub(super) fn new(columns: &'a [Arc]) -> Self { - Self { columns } - } - - /// The newest row position of the tuple `values` (in primary-key order) - /// visible at `max_visible_row`, or `None` if no version is visible. - /// - /// `values` must have one entry per primary-key column. - pub fn get_newest_visible( - &self, - values: &[ScalarValue], - max_visible_row: RowPosition, - ) -> Option { - match self.columns { - [] => None, - // Single-column primary key: one seek-and-stop, no intersection. - [only] => only.get_newest_visible(&values[0], max_visible_row), - // Composite: walk the leading column's visible positions newest-first - // and keep the first that every other column also holds at the same - // physical position (⇒ the whole tuple matches that row). - [leading, rest @ ..] => { - let mut positions = leading.visible_positions(&values[0], max_visible_row); - positions.reverse(); - positions.into_iter().find(|&position| { - rest.iter() - .zip(&values[1..]) - .all(|(column, value)| column.contains_position(value, position)) - }) - } - } - } - - /// Whether `position` is the newest visible row of `values` — the recency - /// check the active index-search arms apply to drop predicate-crossing - /// stale hits. - pub fn is_newest( - &self, - values: &[ScalarValue], - position: RowPosition, - max_visible_row: RowPosition, - ) -> bool { - self.get_newest_visible(values, max_visible_row) == Some(position) - } - - /// Whether `values` has any version visible at `max_visible_row` — the - /// cross-source block-list's existence query, snapshot-bounded so a - /// not-yet-visible write can't shadow an older visible copy. - pub fn contains_visible(&self, values: &[ScalarValue], max_visible_row: RowPosition) -> bool { - self.get_newest_visible(values, max_visible_row).is_some() - } - - /// Whether the primary-key index holds no rows. All columns are inserted - /// together, so the leading column's emptiness answers for the tuple. - pub fn is_empty(&self) -> bool { - self.columns.first().is_none_or(|c| c.is_empty()) - } -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow_array::{Int32Array, RecordBatch, StringArray}; - use arrow_schema::{DataType, Field, Schema}; - - fn btree(field_id: i32, column: &str) -> Arc { - Arc::new(BTreeMemIndex::new(field_id, column.to_string())) - } - - fn composite_batch(ids: &[i32], names: &[&str]) -> RecordBatch { - let schema = Arc::new(Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("name", DataType::Utf8, false), - ])); - RecordBatch::try_new( - schema, - vec![ - Arc::new(Int32Array::from(ids.to_vec())), - Arc::new(StringArray::from(names.to_vec())), - ], - ) - .unwrap() - } - - #[test] - fn single_column_matches_the_btree_directly() { - let id = btree(0, "id"); - // id=1 at positions 0 and 2 (an update), id=2 at position 1. - let b = Int32Array::from(vec![1, 2]); - id.insert( - &RecordBatch::try_new( - Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])), - vec![Arc::new(b)], - ) - .unwrap(), - 0, - ) - .unwrap(); - id.insert( - &RecordBatch::try_new( - Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])), - vec![Arc::new(Int32Array::from(vec![1]))], - ) - .unwrap(), - 2, - ) - .unwrap(); - - let columns = [id]; - let pk = PkLookup::new(&columns); - let one = [ScalarValue::Int32(Some(1))]; - // Watermark above the update sees the newest position; below it, the older. - assert_eq!(pk.get_newest_visible(&one, 5), Some(2)); - assert_eq!(pk.get_newest_visible(&one, 1), Some(0)); - assert!(pk.is_newest(&one, 2, 5)); - assert!(!pk.is_newest(&one, 0, 5)); - } - - #[test] - fn composite_intersects_by_position() { - let id = btree(0, "id"); - let name = btree(1, "name"); - // Rows: (1,"a")@0, (1,"b")@1, (1,"a")@2 — an update of (1,"a"). - let b = composite_batch(&[1, 1, 1], &["a", "b", "a"]); - id.insert(&b, 0).unwrap(); - name.insert(&b, 0).unwrap(); - - let columns = [id, name]; - let pk = PkLookup::new(&columns); - let tuple_1a = [ScalarValue::Int32(Some(1)), ScalarValue::from("a")]; - let tuple_1b = [ScalarValue::Int32(Some(1)), ScalarValue::from("b")]; - - // (1,"a") newest visible row is its re-write at position 2. - assert_eq!(pk.get_newest_visible(&tuple_1a, 5), Some(2)); - // (1,"b") only exists at position 1. - assert_eq!(pk.get_newest_visible(&tuple_1b, 5), Some(1)); - // The stale (1,"a")@0 is not the newest; the re-write @2 is. - assert!(!pk.is_newest(&tuple_1a, 0, 5)); - assert!(pk.is_newest(&tuple_1a, 2, 5)); - // Watermark below the re-write: the older (1,"a")@0 is the newest visible. - assert_eq!(pk.get_newest_visible(&tuple_1a, 1), Some(0)); - assert!(pk.is_newest(&tuple_1a, 0, 1)); - // An absent tuple. - let tuple_2a = [ScalarValue::Int32(Some(2)), ScalarValue::from("a")]; - assert_eq!(pk.get_newest_visible(&tuple_2a, 5), None); - assert!(!pk.contains_visible(&tuple_2a, 5)); - } -} diff --git a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs index 86939ba2eca..04c77b77e28 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs @@ -32,11 +32,9 @@ use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; use crate::session::Session; /// One newer generation's PK membership, used to decide whether it shadows an -/// older source's row. In-memory generations (active / frozen) are **probed by -/// value** against their maintained primary-key BTrees — no per-query set is -/// built — while on-disk generations (flushed, base) carry a materialized -/// PK-hash set (cached by path). The probe is snapshot-bounded, so a -/// not-yet-visible write can't shadow an older visible copy. +/// older source's row. In-memory generations probe their primary-key BTrees by +/// value (snapshot-bounded, so a not-yet-visible write can't shadow an older +/// visible copy); on-disk generations carry a cached PK-hash set. #[derive(Debug, Clone)] pub enum GenMembership { /// Probe the in-memory memtable's index, bounded to its visible prefix. @@ -64,9 +62,7 @@ impl GenMembership { let Some(max) = max_visible_row else { return false; }; - index_store - .pk_index() - .is_some_and(|idx| idx.contains_visible(pk_values, *max)) + index_store.pk_contains_visible(pk_values, *max) } Self::Set(set) => set.contains(&pk_hash), } @@ -87,9 +83,7 @@ impl GenMembership { Self::Index { index_store, max_visible_row, - } => { - max_visible_row.is_none() || index_store.pk_index().is_none_or(|idx| idx.is_empty()) - } + } => max_visible_row.is_none() || index_store.pk_is_empty(), Self::Set(set) => set.is_empty(), } } @@ -216,7 +210,7 @@ fn in_memory_membership( index_store: &Arc, pk_columns: &[String], ) -> Result { - if index_store.pk_index().is_some() { + if index_store.has_pk_index() { let max_visible_row = batch_store.max_visible_row(index_store.max_visible_batch_position()); Ok(GenMembership::Index { index_store: index_store.clone(), diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs index b59b083da02..e1495cb0bb1 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec/newest_pk_filter.rs @@ -11,9 +11,9 @@ //! //! This node closes that hole with a predicate-independent recency check: for //! each hit it asks the memtable's maintained primary-key index -//! ([`crate::dataset::mem_wal::index::PkLookup`]) whether the hit's own row -//! position is the newest version of its primary key visible at the query's -//! `max_visible` watermark, and keeps the hit **iff so**. A stale hit (some +//! ([`IndexStore::pk_is_newest`]) whether the hit's own row position is the +//! newest version of its primary key visible at the query's `max_visible` +//! watermark, and keeps the hit **iff so**. A stale hit (some //! newer version exists) is dropped even when that newer version never appears //! in the result. This is exactly the seek point-lookup already does; the index //! search arms simply didn't do it. @@ -48,7 +48,8 @@ pub struct NewestPkFilterExec { input: Arc, pk_columns: Vec, row_id_column: String, - /// Holds the maintained primary-key index (`PkLookup`) queried per hit. + /// Holds the maintained primary-key index, queried per hit via + /// [`IndexStore::pk_is_newest`]. index_store: Arc, /// Resolves the `max_visible` row watermark from the visible batch prefix. batch_store: Arc, @@ -197,9 +198,9 @@ impl NewestPkFilterStream { fn filter_batch(&self, batch: RecordBatch) -> DFResult { // No primary-key index (memtable without a primary key), no visible // rows, or an empty batch: nothing to dedup against, so pass it through. - let Some(pk_index) = self.index_store.pk_index() else { + if !self.index_store.has_pk_index() { return Ok(batch); - }; + } let Some(max_visible_row) = self.max_visible_row else { return Ok(batch); }; @@ -239,7 +240,10 @@ impl NewestPkFilterStream { .map(|&col| ScalarValue::try_from_array(batch.column(col), row)) .collect::>()?; // Keep iff this hit is the newest visible version of its PK. - keep.push(pk_index.is_newest(&values, position, max_visible_row)); + keep.push( + self.index_store + .pk_is_newest(&values, position, max_visible_row), + ); } filter_record_batch(&batch, &BooleanArray::from(keep)) .map_err(|e| DataFusionError::ArrowError(Box::new(e), None)) diff --git a/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs b/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs index 1107cac1bfc..7bd51518a6c 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/point_lookup.rs @@ -669,12 +669,10 @@ fn probe_position( } let max_visible_row = visible_end - 1; - // A single-column primary key always has a value-keyed BTree on it (a - // reused user index or the auto-created `__pk__*` one — see - // `IndexStore::enable_pk_index`). It is collision-free, so one seek-and-stop - // yields the answer with no value re-check. Absent only when the table has - // no primary-key index at all, in which case the caller falls back to the - // plan path. + // A single-column primary key always has a value-keyed BTree (reused or + // auto-created — see `IndexStore::enable_pk_index`): collision-free, so one + // seek yields the answer with no re-check. Absent only when the table has no + // PK index, where the caller falls back to the plan path. let Some(btree) = index_store.get_btree_by_column(pk_column) else { return Ok(ProbePos::NoIndex); }; From bc115669c5bf00d5ba2f4737e85fb373ec2e72e5 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Wed, 3 Jun 2026 14:05:28 -0500 Subject: [PATCH 08/12] docs(mem_wal): fix broken intra-doc link to pk_newest_visible The enable_pk_index doc referenced the removed `Self::pk_index`, failing `cargo doc -D warnings`. Co-Authored-By: Claude Opus 4.8 (1M context) --- rust/lance/src/dataset/mem_wal/index.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/lance/src/dataset/mem_wal/index.rs b/rust/lance/src/dataset/mem_wal/index.rs index 0d540bfdf48..64051c3efab 100644 --- a/rust/lance/src/dataset/mem_wal/index.rs +++ b/rust/lance/src/dataset/mem_wal/index.rs @@ -381,7 +381,7 @@ impl IndexStore { } /// Maintain a BTree on each primary-key column so the memtable can answer - /// "newest visible version of this key" (see [`Self::pk_index`]). + /// "newest visible version of this key" (see [`Self::pk_newest_visible`]). /// /// Reuses an existing BTree on the field, else auto-creates one under a /// `__pk__*` name so the normal insert loop maintains it. Call once at From 7e83d81d732f08e9e745b621bd8c6c70fe7da163 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Thu, 4 Jun 2026 05:36:57 -0500 Subject: [PATCH 09/12] refactor(mem_wal): composite-key PK BTree + standalone on-disk dedup index MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Implements jackye1995's PR #7067 review: maintain the primary key as a true LSM BTree end to end instead of per-column intersection + materialized PK-hash sets. In-memory: replace `pk_btrees: Vec` and the position-intersection walk with a single `PkIndex` — `Single` (arity 1, reuses the column BTree, no second skiplist) or `Composite` (arity >=2, one order-preserving encoded-tuple skiplist; new `index/pk_key.rs`). `pk_newest_visible` is now one seek for any arity. Drop `BTreeMemIndex::visible_positions`/`contains_position`. On disk: flush writes a standalone PK BTree sidecar at `{gen}/_pk_index` via `train_btree_index` (not a manifest index, no synthetic data column). The block-list opens it by path and probes with `Equals`; the opened index and its pages ride the injected `FlushedMemTableCache`/`LanceCache`, so steady-state probes are memory-resident with no upfront PK-column scan. Drop `GenMembership::Set`, `scan_pk_hashes`, `pk_hashes_from_batch*`, and `needs_values`; membership is now `InMemory` (sync value probe) or `OnDisk` (async `Equals`). Rename `PkHashFilterExec` -> `PkBlockFilterExec` (async in-flight-future stream). Repurpose `FlushedMemTableCache.pk_hashes` to cache the opened `ScalarIndex`. Net deletion. mem_wal suite green; clippy --tests --benches -D warnings clean. Co-Authored-By: Claude Opus 4.8 (1M context) --- rust/lance/src/dataset/mem_wal/index.rs | 200 ++++-- rust/lance/src/dataset/mem_wal/index/btree.rs | 103 --- .../lance/src/dataset/mem_wal/index/pk_key.rs | 348 +++++++++ .../src/dataset/mem_wal/memtable/flush.rs | 148 ++++ .../src/dataset/mem_wal/scanner/block_list.rs | 671 +++++++----------- .../src/dataset/mem_wal/scanner/builder.rs | 41 +- .../lance/src/dataset/mem_wal/scanner/exec.rs | 6 +- .../src/dataset/mem_wal/scanner/exec/pk.rs | 2 +- .../mem_wal/scanner/exec/pk_block_filter.rs | 348 +++++++++ .../mem_wal/scanner/exec/pk_hash_filter.rs | 366 ---------- .../dataset/mem_wal/scanner/flushed_cache.rs | 73 +- .../src/dataset/mem_wal/scanner/planner.rs | 99 ++- .../dataset/mem_wal/scanner/vector_search.rs | 36 +- rust/lance/src/dataset/mem_wal/util.rs | 10 + 14 files changed, 1406 insertions(+), 1045 deletions(-) create mode 100644 rust/lance/src/dataset/mem_wal/index/pk_key.rs create mode 100644 rust/lance/src/dataset/mem_wal/scanner/exec/pk_block_filter.rs delete mode 100644 rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs diff --git a/rust/lance/src/dataset/mem_wal/index.rs b/rust/lance/src/dataset/mem_wal/index.rs index 64051c3efab..c3cce7525e7 100644 --- a/rust/lance/src/dataset/mem_wal/index.rs +++ b/rust/lance/src/dataset/mem_wal/index.rs @@ -18,6 +18,7 @@ mod arena_skiplist; mod btree; mod fts; mod hnsw; +mod pk_key; use std::collections::HashMap; use std::sync::Arc; @@ -47,6 +48,26 @@ pub type RowPosition = u64; pub use btree::{BTreeIndexConfig, BTreeMemIndex}; pub use fts::{FtsIndexConfig, FtsMemIndex, FtsQueryExpr, SearchOptions}; pub use hnsw::{HnswIndexConfig, HnswMemIndex}; +pub use pk_key::encode_pk_tuple; + +use pk_key::PkKeyIndex; + +/// The memtable's primary-key index, used to answer "newest visible version of +/// this key" for dedup. Single-column PKs reuse the column's compact typed +/// [`BTreeMemIndex`] (no second copy); composite PKs use one [`PkKeyIndex`] over +/// the order-preserving encoded tuple ([`encode_pk_tuple`]). Either way the +/// lookup is a single seek. +enum PkIndex { + /// Arity 1: aliases a `btree_indexes` entry, so the insert loop maintains it. + Single(Arc), + /// Arity >= 2: a dedicated encoded-tuple skiplist, maintained explicitly in + /// the insert paths. `columns` are the PK columns in order, resolved against + /// each batch's schema at insert time. + Composite { + index: Arc, + columns: Vec, + }, +} // ============================================================================ // Index Store @@ -205,11 +226,10 @@ pub struct IndexStore { hnsw_indexes: HashMap, /// FTS indexes keyed by index name. fts_indexes: HashMap, - /// The primary-key BTrees, one per PK column in order. Each aliases a - /// `btree_indexes` entry (reused or auto-created), so the insert loop - /// maintains it. Empty without a primary key; queried via - /// [`Self::pk_newest_visible`] (see [`Self::enable_pk_index`]). - pk_btrees: Vec>, + /// The primary-key index (single-column or composite), or `None` without a + /// primary key. Queried via [`Self::pk_newest_visible`] (see + /// [`Self::enable_pk_index`]). + pk_index: Option, /// Maximum batch position that is durable in the WAL and therefore /// visible to scanners. Advanced unconditionally after a WAL append /// succeeds; not gated on whether any indexes are configured. @@ -222,7 +242,7 @@ impl Default for IndexStore { btree_indexes: HashMap::new(), hnsw_indexes: HashMap::new(), fts_indexes: HashMap::new(), - pk_btrees: Vec::new(), + pk_index: None, max_visible_batch_position: AtomicUsize::new(0), } } @@ -241,12 +261,14 @@ impl std::fmt::Debug for IndexStore { ) .field("fts_indexes", &self.fts_indexes.keys().collect::>()) .field( - "pk_btrees", - &self - .pk_btrees - .iter() - .map(|b| b.column_name()) - .collect::>(), + "pk_index", + &match &self.pk_index { + None => "none".to_string(), + Some(PkIndex::Single(b)) => format!("single({})", b.column_name()), + Some(PkIndex::Composite { columns, .. }) => { + format!("composite({})", columns.join(", ")) + } + }, ) .field( "max_visible_batch_position", @@ -380,62 +402,101 @@ impl IndexStore { .insert(name, FtsMemIndex::with_params(field_id, column, params)); } - /// Maintain a BTree on each primary-key column so the memtable can answer - /// "newest visible version of this key" (see [`Self::pk_newest_visible`]). + /// Maintain a primary-key index so the memtable can answer "newest visible + /// version of this key" (see [`Self::pk_newest_visible`]). /// - /// Reuses an existing BTree on the field, else auto-creates one under a - /// `__pk__*` name so the normal insert loop maintains it. Call once at + /// Single-column PKs reuse an existing BTree on the field, else auto-create + /// one under a `__pk__*` name so the normal insert loop maintains it (no + /// second copy). Composite (arity >= 2) PKs build a dedicated encoded-tuple + /// [`PkKeyIndex`], maintained explicitly in the insert paths. Call once at /// construction, after [`Self::from_configs`] and before any inserts; a - /// no-op when `pk_columns` is empty. The auto-created BTrees are in-memory - /// only — flush rebuilds from the user's index configs — so a primary key - /// without a user scalar index gains no on-disk index. + /// no-op when `pk_columns` is empty. pub fn enable_pk_index(&mut self, pk_columns: &[(String, i32)]) { - for (column, field_id) in pk_columns { - let btree = match self - .btree_indexes - .values() - .find(|b| b.field_id() == *field_id) - { - Some(existing) => existing.clone(), - None => { - let btree = Arc::new(BTreeMemIndex::new(*field_id, column.clone())); - self.btree_indexes - .insert(format!("__pk__{column}"), btree.clone()); - btree - } - }; - self.pk_btrees.push(btree); - } + self.pk_index = match pk_columns { + [] => None, + [(column, field_id)] => { + let btree = match self + .btree_indexes + .values() + .find(|b| b.field_id() == *field_id) + { + Some(existing) => existing.clone(), + None => { + let btree = Arc::new(BTreeMemIndex::new(*field_id, column.clone())); + self.btree_indexes + .insert(format!("__pk__{column}"), btree.clone()); + btree + } + }; + Some(PkIndex::Single(btree)) + } + multi => Some(PkIndex::Composite { + index: Arc::new(PkKeyIndex::new()), + columns: multi.iter().map(|(c, _)| c.clone()).collect(), + }), + }; } - /// Whether the memtable has a primary-key index (a BTree per PK column). + /// Whether the memtable has a primary-key index. pub fn has_pk_index(&self) -> bool { - !self.pk_btrees.is_empty() + self.pk_index.is_some() + } + + /// Sorted `(value, row_id)` training batches for the flushed on-disk PK + /// BTree (the sidecar dedup index). Single-column emits the typed PK value; + /// composite emits the order-preserving `Binary` encoded tuple. Empty when + /// there is no primary key. Row positions line up 1:1 with the forward- + /// written data file, so they are the flushed row ids directly. + pub fn pk_training_batches(&self, batch_size: usize) -> Result> { + match &self.pk_index { + None => Ok(Vec::new()), + Some(PkIndex::Single(btree)) => btree.to_training_batches(batch_size), + Some(PkIndex::Composite { index, .. }) => index.to_training_batches(batch_size), + } + } + + /// Resolve the PK columns' positions in `batch` (composite insert helper). + fn pk_batch_indices(batch: &RecordBatch, columns: &[String]) -> Result> { + columns + .iter() + .map(|c| { + batch + .schema() + .column_with_name(c) + .map(|(i, _)| i) + .ok_or_else(|| { + Error::invalid_input(format!("PK column '{c}' not found in batch")) + }) + }) + .collect() + } + + /// Maintain the composite PK index for `batch` (no-op for single/no PK). + fn insert_composite_pk(&self, batch: &RecordBatch, row_offset: u64) -> Result<()> { + if let Some(PkIndex::Composite { index, columns }) = &self.pk_index { + let pk_indices = Self::pk_batch_indices(batch, columns)?; + index.insert(batch, &pk_indices, row_offset)?; + } + Ok(()) } /// The newest row position of the primary-key tuple `values` (in PK order) - /// visible at `max_visible_row`, or `None`. - /// - /// Single-column is one seek. A composite key walks the leading column's - /// visible positions newest-first and returns the first that every other - /// column also holds at the same physical position — so the whole tuple - /// matches that row. Collision-free, since `position` is the row identity. + /// visible at `max_visible_row`, or `None`. A single seek either way: + /// single-column probes the typed BTree; composite probes the encoded-tuple + /// index. Collision-free, since `position` is the row identity. pub fn pk_newest_visible( &self, values: &[ScalarValue], max_visible_row: RowPosition, ) -> Option { - match self.pk_btrees.as_slice() { - [] => None, - [only] => only.get_newest_visible(&values[0], max_visible_row), - [leading, rest @ ..] => { - let mut positions = leading.visible_positions(&values[0], max_visible_row); - positions.reverse(); - positions.into_iter().find(|&position| { - rest.iter() - .zip(&values[1..]) - .all(|(column, value)| column.contains_position(value, position)) - }) + match &self.pk_index { + None => None, + Some(PkIndex::Single(btree)) => btree.get_newest_visible(&values[0], max_visible_row), + Some(PkIndex::Composite { index, .. }) => { + // An unsupported PK type would have failed at insert, so the + // index can't hold a tuple this fails to encode. + let key = encode_pk_tuple(values).ok()?; + index.get_newest_visible(&key, max_visible_row) } } } @@ -464,11 +525,13 @@ impl IndexStore { self.pk_newest_visible(values, max_visible_row).is_some() } - /// Whether the primary-key index holds no rows (or doesn't exist). All - /// columns are inserted together, so the leading column answers for the - /// tuple. + /// Whether the primary-key index holds no rows (or doesn't exist). pub fn pk_is_empty(&self) -> bool { - self.pk_btrees.first().is_none_or(|c| c.is_empty()) + match &self.pk_index { + None => true, + Some(PkIndex::Single(btree)) => btree.is_empty(), + Some(PkIndex::Composite { index, .. }) => index.is_empty(), + } } /// Insert a batch into all indexes. @@ -493,7 +556,9 @@ impl IndexStore { for index in self.fts_indexes.values() { index.insert(batch, row_offset)?; } - // PK BTrees alias `btree_indexes` entries — already maintained above. + // Single-column PK aliases a `btree_indexes` entry (maintained above); + // a composite PK has its own index, maintained here. + self.insert_composite_pk(batch, row_offset)?; // Update global watermark after all indexes have been updated if let Some(bp) = batch_position { @@ -550,7 +615,11 @@ impl IndexStore { } } - // PK BTrees alias `btree_indexes` entries — already maintained above. + // Single-column PK aliases a `btree_indexes` entry (maintained above); + // a composite PK has its own index, maintained here. + for stored in batches { + self.insert_composite_pk(&stored.data, stored.row_offset)?; + } // Update global watermark to the max batch position let max_bp = batches.iter().map(|b| b.batch_position).max().unwrap(); @@ -664,8 +733,13 @@ impl IndexStore { .map(|(name, _idx_type, duration)| (name.to_string(), duration)) .collect(); - // PK BTrees alias `btree_indexes` entries — their threads above - // already maintained them (and joined before the watermark advances). + // Single-column PK aliases a `btree_indexes` entry — its thread above + // already maintained it (and joined). A composite PK has its own + // index; maintain it here before the watermark advances so the + // visible prefix is fully indexed. + for stored in batches { + self.insert_composite_pk(&stored.data, stored.row_offset)?; + } // Update global watermark to the max batch position let max_bp = batches.iter().map(|b| b.batch_position).max().unwrap(); @@ -843,7 +917,7 @@ mod tests { } #[test] - fn pk_newest_visible_composite_intersects_by_position() { + fn pk_newest_visible_composite_seeks_encoded_tuple() { let mut store = IndexStore::new(); store.enable_pk_index(&[("id".to_string(), 0), ("name".to_string(), 1)]); // Rows: (1,"a")@0, (1,"b")@1, (1,"a")@2 — an update of (1,"a"). diff --git a/rust/lance/src/dataset/mem_wal/index/btree.rs b/rust/lance/src/dataset/mem_wal/index/btree.rs index e0123110f43..c2f89b9932d 100644 --- a/rust/lance/src/dataset/mem_wal/index/btree.rs +++ b/rust/lance/src/dataset/mem_wal/index/btree.rs @@ -398,34 +398,6 @@ impl FixedIntBackend { positions } - fn visible_positions(&self, value: &ScalarValue, max: RowPosition) -> Vec { - let Some(enc) = encode_scalar(value) else { - if value.is_null() { - let mut positions: Vec = self - .null_positions - .lock() - .unwrap() - .iter() - .copied() - .filter(|p| *p <= max) - .collect(); - positions.sort_unstable(); - return positions; - } - return Vec::new(); - }; - let start = FixedKey { enc, position: 0 }; - let mut positions = Vec::new(); - // For a fixed enc, positions ascend — stop at the first past the watermark. - for key in self.reader.range_from(&start) { - if key.enc != enc || key.position > max { - break; - } - positions.push(key.position); - } - positions - } - fn len(&self) -> usize { self.reader.len() + self.null_positions.lock().unwrap().len() } @@ -582,36 +554,6 @@ impl BytesBackend { positions } - fn visible_positions(&self, value: &ScalarValue, max: RowPosition) -> Vec { - let Some(bytes) = value_bytes(value) else { - if value.is_null() { - let mut positions: Vec = self - .null_positions - .lock() - .unwrap() - .iter() - .copied() - .filter(|p| *p <= max) - .collect(); - positions.sort_unstable(); - return positions; - } - return Vec::new(); - }; - let start = BytesKey { - bytes: InlineBytes::new(bytes), - position: 0, - }; - let mut positions = Vec::new(); - for key in self.reader.range_from(&start) { - if key.bytes.as_slice() != bytes || key.position > max { - break; - } - positions.push(key.position); - } - positions - } - fn len(&self) -> usize { self.reader.len() + self.null_positions.lock().unwrap().len() } @@ -786,21 +728,6 @@ impl ScalarBackend { positions } - fn visible_positions(&self, value: &ScalarValue, max: RowPosition) -> Vec { - let start = IndexKey { - value: OrderableScalarValue(value.clone()), - row_position: 0, - }; - let mut positions = Vec::new(); - for key in self.reader.range_from(&start) { - if key.value.0 != *value || key.row_position > max { - break; - } - positions.push(key.row_position); - } - positions - } - fn len(&self) -> usize { self.reader.len() } @@ -866,14 +793,6 @@ impl Backend { } } - fn visible_positions(&self, value: &ScalarValue, max: RowPosition) -> Vec { - match self { - Self::FixedInt(b) => b.visible_positions(value, max), - Self::Bytes(b) => b.visible_positions(value, max), - Self::Scalar(b) => b.visible_positions(value, max), - } - } - fn len(&self) -> usize { match self { Self::FixedInt(b) => b.len(), @@ -974,28 +893,6 @@ impl BTreeMemIndex { self.backend.get().map(|b| b.get(value)).unwrap_or_default() } - /// All row positions for `value` that are `<= max_visible_row`, ascending. - /// The composite-primary-key intersection driver (see - /// [`super::IndexStore::pk_newest_visible`]). - pub fn visible_positions( - &self, - value: &ScalarValue, - max_visible_row: RowPosition, - ) -> Vec { - self.backend - .get() - .map(|b| b.visible_positions(value, max_visible_row)) - .unwrap_or_default() - } - - /// Whether the exact `(value, position)` entry exists — one seek (the newest - /// entry for `value` at or below `position` equals `position` iff present). - /// Probes a candidate position against the other PK columns (see - /// [`super::IndexStore::pk_newest_visible`]). - pub fn contains_position(&self, value: &ScalarValue, position: RowPosition) -> bool { - self.get_newest_visible(value, position) == Some(position) - } - /// Get the number of entries (not unique values). pub fn len(&self) -> usize { self.backend.get().map(|b| b.len()).unwrap_or(0) diff --git a/rust/lance/src/dataset/mem_wal/index/pk_key.rs b/rust/lance/src/dataset/mem_wal/index/pk_key.rs new file mode 100644 index 00000000000..ce27f30b355 --- /dev/null +++ b/rust/lance/src/dataset/mem_wal/index/pk_key.rs @@ -0,0 +1,348 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright The Lance Authors + +//! Composite primary-key encoding + index for MemWAL dedup. +//! +//! A multi-column primary key is reduced to a single order-preserving byte +//! string ([`encode_pk_tuple`]) so the whole tuple is one comparable key: +//! lexicographic byte order equals tuple order, and distinct tuples never +//! collide. The same key drives both the in-memory composite index +//! ([`PkKeyIndex`], a skiplist) and the flushed on-disk BTree (the key is the +//! index's `Binary` value column), so a probe builds `ScalarValue::Binary(key)` +//! and both ends agree. +//! +//! Single-column primary keys do **not** use this — they keep the compact typed +//! BTree directly (see [`super::BTreeMemIndex`]). + +use std::sync::Mutex; + +use arrow_array::{BinaryArray, RecordBatch, UInt64Array}; +use arrow_schema::{DataType, Field, Schema}; +use datafusion::common::ScalarValue; +use lance_core::{Error, ROW_ID, Result}; +use lance_index::scalar::registry::VALUE_COLUMN_NAME; +use std::sync::Arc; + +use super::RowPosition; +use super::arena_skiplist::{SkipListReader, SkipListWriter, new_skiplist}; + +/// Sign-flip a signed integer to an order-preserving unsigned key (matches the +/// fixed-int BTree backend). Big-endian bytes of the result sort like the value. +#[inline] +fn encode_signed(v: i64) -> u64 { + (v as u64) ^ (1u64 << 63) +} + +/// Append an order-preserving encoding of one non-null byte string: each `0x00` +/// is escaped to `0x00 0xFF`, then a `0x00 0x00` terminator is appended. The +/// terminator sorts before any escaped content, so a prefix orders before its +/// extensions and no value can forge a column boundary. +fn encode_bytes(out: &mut Vec, bytes: &[u8]) { + for &b in bytes { + out.push(b); + if b == 0x00 { + out.push(0xFF); + } + } + out.extend_from_slice(&[0x00, 0x00]); +} + +/// Append the order-preserving encoding of a single PK column value. A leading +/// tag (`0x00` null / `0x01` non-null) makes nulls sort first and keeps the +/// per-column encoding self-delimiting (fixed-width for ints, terminated for +/// bytes), so concatenating columns stays injective and order-preserving. +fn encode_value(out: &mut Vec, value: &ScalarValue) -> Result<()> { + if value.is_null() { + out.push(0x00); + return Ok(()); + } + out.push(0x01); + macro_rules! be_signed { + ($v:expr) => { + out.extend_from_slice(&encode_signed($v as i64).to_be_bytes()) + }; + } + match value { + ScalarValue::Int8(Some(v)) => be_signed!(*v), + ScalarValue::Int16(Some(v)) => be_signed!(*v), + ScalarValue::Int32(Some(v)) => be_signed!(*v), + ScalarValue::Int64(Some(v)) => be_signed!(*v), + ScalarValue::Date32(Some(v)) => be_signed!(*v), + ScalarValue::Date64(Some(v)) => be_signed!(*v), + ScalarValue::UInt8(Some(v)) => out.extend_from_slice(&(*v as u64).to_be_bytes()), + ScalarValue::UInt16(Some(v)) => out.extend_from_slice(&(*v as u64).to_be_bytes()), + ScalarValue::UInt32(Some(v)) => out.extend_from_slice(&(*v as u64).to_be_bytes()), + ScalarValue::UInt64(Some(v)) => out.extend_from_slice(&v.to_be_bytes()), + ScalarValue::Boolean(Some(b)) => out.push(*b as u8), + ScalarValue::Utf8(Some(s)) | ScalarValue::LargeUtf8(Some(s)) => { + encode_bytes(out, s.as_bytes()) + } + ScalarValue::Binary(Some(b)) + | ScalarValue::LargeBinary(Some(b)) + | ScalarValue::FixedSizeBinary(_, Some(b)) => encode_bytes(out, b), + other => { + return Err(Error::invalid_input(format!( + "Unsupported primary-key column type for composite key: {other:?}" + ))); + } + } + Ok(()) +} + +/// Encode a PK tuple (values in PK column order) to one order-preserving key. +pub fn encode_pk_tuple(values: &[ScalarValue]) -> Result> { + let mut out = Vec::with_capacity(values.len() * 9); + for value in values { + encode_value(&mut out, value)?; + } + Ok(out) +} + +/// Encode row `row` of `batch`'s PK columns (at `pk_indices`) to one key. +fn encode_pk_row(batch: &RecordBatch, pk_indices: &[usize], row: usize) -> Result> { + let mut out = Vec::with_capacity(pk_indices.len() * 9); + for &col in pk_indices { + let value = ScalarValue::try_from_array(batch.column(col), row)?; + encode_value(&mut out, &value)?; + } + Ok(out) +} + +/// Skiplist key: the encoded tuple plus the row position (makes every entry +/// unique, so a non-unique key keeps every version). Sorts by `(bytes, pos)`. +#[derive(PartialEq, Eq)] +struct EncodedKey { + bytes: Box<[u8]>, + position: RowPosition, +} + +impl PartialOrd for EncodedKey { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for EncodedKey { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + self.bytes + .cmp(&other.bytes) + .then(self.position.cmp(&other.position)) + } +} + +/// In-memory composite primary-key index: a skiplist over `(encoded_tuple, +/// position)`. Answers "newest visible version of this tuple" in one seek, the +/// composite analogue of [`super::BTreeMemIndex::get_newest_visible`]. +pub struct PkKeyIndex { + reader: SkipListReader, + writer: Mutex>, +} + +impl std::fmt::Debug for PkKeyIndex { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("PkKeyIndex") + .field("len", &self.len()) + .finish() + } +} + +impl PkKeyIndex { + pub fn new() -> Self { + let (writer, reader) = new_skiplist::(); + Self { + reader, + writer: Mutex::new(writer), + } + } + + /// Insert every row of `batch`, encoding the PK columns at `pk_indices`. + /// `row_offset` is the absolute position of the first row. + pub fn insert(&self, batch: &RecordBatch, pk_indices: &[usize], row_offset: u64) -> Result<()> { + let mut writer = self.writer.lock().unwrap(); + for row in 0..batch.num_rows() { + let bytes = encode_pk_row(batch, pk_indices, row)?; + writer.insert(EncodedKey { + bytes: bytes.into_boxed_slice(), + position: row_offset + row as u64, + }); + } + Ok(()) + } + + /// Newest position of the pre-encoded tuple `key` visible at + /// `max_visible_row`, or `None`. A single seek-and-stop (no allocation). + pub fn get_newest_visible( + &self, + key: &[u8], + max_visible_row: RowPosition, + ) -> Option { + let target = EncodedKey { + bytes: key.into(), + position: max_visible_row, + }; + self.reader + .upper_bound_with(&target, |found| { + (found.bytes.as_ref() == key).then_some(found.position) + }) + .flatten() + } + + pub fn len(&self) -> usize { + self.reader.len() + } + + pub fn is_empty(&self) -> bool { + self.reader.len() == 0 + } + + /// Export as sorted `(Binary value, row_id)` batches to train the flushed + /// on-disk BTree. Entries are already in `(bytes, position)` order, so the + /// stream is sorted by value as `train_btree_index` requires. + pub fn to_training_batches(&self, batch_size: usize) -> Result> { + let schema = Arc::new(Schema::new(vec![ + Field::new(VALUE_COLUMN_NAME, DataType::Binary, true), + Field::new(ROW_ID, DataType::UInt64, false), + ])); + + let mut batches = Vec::new(); + let mut keys: Vec> = Vec::with_capacity(batch_size); + let mut row_ids: Vec = Vec::with_capacity(batch_size); + for entry in self.reader.iter() { + keys.push(entry.bytes.to_vec()); + row_ids.push(entry.position); + if keys.len() >= batch_size { + batches.push(build_batch(&schema, &keys, &row_ids)?); + keys.clear(); + row_ids.clear(); + } + } + if !keys.is_empty() { + batches.push(build_batch(&schema, &keys, &row_ids)?); + } + Ok(batches) + } +} + +fn build_batch(schema: &Arc, keys: &[Vec], row_ids: &[u64]) -> Result { + let values = BinaryArray::from_iter_values(keys.iter()); + let ids = UInt64Array::from(row_ids.to_vec()); + RecordBatch::try_new(schema.clone(), vec![Arc::new(values), Arc::new(ids)]) + .map_err(|e| Error::io(e.to_string())) +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow_array::{Int32Array, StringArray}; + + fn tuple(a: i32, b: &str) -> Vec { + vec![ScalarValue::Int32(Some(a)), ScalarValue::from(b)] + } + + #[test] + fn encoding_is_order_preserving_and_injective() { + // Sorting tuples by their encoding must match tuple order, and distinct + // tuples must produce distinct bytes. + let tuples = [ + tuple(1, "a"), + tuple(1, "ab"), + tuple(1, "b"), + tuple(2, "a"), + tuple(-1, "z"), + ]; + let mut encoded: Vec<(Vec, &Vec)> = tuples + .iter() + .map(|t| (encode_pk_tuple(t).unwrap(), t)) + .collect(); + encoded.sort_by(|x, y| x.0.cmp(&y.0)); + let order: Vec<_> = encoded.iter().map(|(_, t)| (*t).clone()).collect(); + // -1 < 1 < 2; within id=1, "a" < "ab" < "b". + assert_eq!( + order, + vec![ + tuple(-1, "z"), + tuple(1, "a"), + tuple(1, "ab"), + tuple(1, "b"), + tuple(2, "a"), + ] + ); + // Injective: 5 distinct tuples → 5 distinct keys. + let mut keys: Vec> = tuples.iter().map(|t| encode_pk_tuple(t).unwrap()).collect(); + keys.sort(); + keys.dedup(); + assert_eq!(keys.len(), 5); + } + + #[test] + fn null_sorts_first_and_is_distinct() { + let null_a = vec![ScalarValue::Int32(None), ScalarValue::from("a")]; + let one_a = tuple(1, "a"); + assert!(encode_pk_tuple(&null_a).unwrap() < encode_pk_tuple(&one_a).unwrap()); + assert_ne!( + encode_pk_tuple(&null_a).unwrap(), + encode_pk_tuple(&one_a).unwrap() + ); + } + + #[test] + fn prefix_safety_with_embedded_zero() { + // A string containing 0x00 must not collide with or sort incorrectly + // against a shorter one (escaping + terminator). + let with_zero = vec![ScalarValue::Binary(Some(vec![0x00]))]; + let empty = vec![ScalarValue::Binary(Some(vec![]))]; + assert!(encode_pk_tuple(&empty).unwrap() < encode_pk_tuple(&with_zero).unwrap()); + } + + fn id_name_batch(ids: &[i32], names: &[&str]) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + RecordBatch::try_new( + schema, + vec![ + Arc::new(Int32Array::from(ids.to_vec())), + Arc::new(StringArray::from(names.to_vec())), + ], + ) + .unwrap() + } + + #[test] + fn pk_key_index_newest_visible_is_snapshot_bounded() { + let index = PkKeyIndex::new(); + // (1,"a")@0, (1,"b")@1, (1,"a")@2 — an update of (1,"a"). + index + .insert(&id_name_batch(&[1, 1, 1], &["a", "b", "a"]), &[0, 1], 0) + .unwrap(); + + let key_1a = encode_pk_tuple(&tuple(1, "a")).unwrap(); + let key_1b = encode_pk_tuple(&tuple(1, "b")).unwrap(); + // Newest visible (1,"a") is its re-write at position 2... + assert_eq!(index.get_newest_visible(&key_1a, 5), Some(2)); + // ...but bounded below the re-write, the older copy at 0. + assert_eq!(index.get_newest_visible(&key_1a, 1), Some(0)); + assert_eq!(index.get_newest_visible(&key_1b, 5), Some(1)); + // Absent tuple. + let key_2a = encode_pk_tuple(&tuple(2, "a")).unwrap(); + assert_eq!(index.get_newest_visible(&key_2a, 5), None); + } + + #[test] + fn training_batches_are_value_sorted() { + let index = PkKeyIndex::new(); + index + .insert(&id_name_batch(&[2, 1], &["a", "b"]), &[0, 1], 0) + .unwrap(); + let batches = index.to_training_batches(8192).unwrap(); + assert_eq!(batches.len(), 1); + let values = batches[0] + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + // (1,"b") encodes below (2,"a"), so it comes first. + assert!(values.value(0) < values.value(1)); + } +} diff --git a/rust/lance/src/dataset/mem_wal/memtable/flush.rs b/rust/lance/src/dataset/mem_wal/memtable/flush.rs index f8e9e7db1d6..b1527472528 100644 --- a/rust/lance/src/dataset/mem_wal/memtable/flush.rs +++ b/rust/lance/src/dataset/mem_wal/memtable/flush.rs @@ -427,6 +427,10 @@ impl MemTableFlusher { all_indexes.extend(fts_indexes); } + // Write the standalone primary-key dedup index (sidecar, not a manifest + // index — the block-list opens it directly by path). + self.create_pk_index(&gen_path, memtable.indexes()).await?; + // Write a single manifest that records the fragments, the // within-generation deletion vector, and all indexes, overwriting the // data-only v1 manifest created by Dataset::write. @@ -521,6 +525,49 @@ impl MemTableFlusher { Ok(created_indexes) } + /// Write the standalone primary-key dedup index for this generation. + /// + /// Unlike user indexes, this is a **sidecar**: it is not registered in the + /// manifest. The block-list opens it directly by path + /// ([`pk_index_path`]) and probes it with `Equals`. Single-column primary + /// keys index the typed value; composite keys index the order-preserving + /// `Binary` encoded tuple (see [`super::super::index::encode_pk_tuple`]). + /// Row positions line up 1:1 with the forward-written data file, so they are + /// the flushed row ids directly. No-op without a primary-key index. + async fn create_pk_index( + &self, + gen_path: &Path, + mem_indexes: Option<&super::super::index::IndexStore>, + ) -> Result<()> { + use datafusion::physical_plan::SendableRecordBatchStream; + use datafusion::physical_plan::stream::RecordBatchStreamAdapter; + use lance_index::scalar::btree::train_btree_index; + use lance_index::scalar::lance_format::LanceIndexStore; + + use crate::dataset::mem_wal::util::pk_index_path; + + let Some(registry) = mem_indexes else { + return Ok(()); + }; + let batches = registry.pk_training_batches(8192)?; + if batches.is_empty() { + return Ok(()); + } + + let schema = batches[0].schema(); + let store = LanceIndexStore::new( + self.object_store.clone(), + pk_index_path(gen_path), + Arc::new(LanceCache::no_cache()), + ); + let stream: SendableRecordBatchStream = Box::pin(RecordBatchStreamAdapter::new( + schema, + futures::stream::iter(batches.into_iter().map(Ok)), + )); + train_btree_index(stream, &store, 8192, None, None).await?; + Ok(()) + } + /// Create FTS (Full-Text Search) indexes from in-memory data (uncommitted). /// /// Writes the FTS index files and returns index metadata without committing. @@ -1170,6 +1217,107 @@ mod tests { assert_eq!(rows.get(&3), Some(&"c2".to_string())); } + /// Flushing a memtable with a primary-key index writes a standalone sidecar + /// BTree at `{gen}/_pk_index` that the block-list can reopen by path and + /// probe by value — including for a within-gen-superseded PK (existence, + /// not visibility). + #[tokio::test] + async fn flushed_pk_index_sidecar_is_probeable() { + use lance_core::cache::LanceCache; + use lance_index::metrics::NoOpMetricsCollector; + use lance_index::registry::IndexPluginRegistry; + use lance_index::scalar::lance_format::LanceIndexStore; + use lance_index::scalar::{SargableQuery, SearchResult}; + + use super::super::super::index::IndexStore; + use crate::dataset::mem_wal::util::pk_index_path; + use datafusion::common::ScalarValue; + + let (store, base_path, _base_uri, _temp_dir) = create_local_store().await; + let shard_id = Uuid::new_v4(); + let manifest_store = Arc::new(ShardManifestStore::new( + store.clone(), + &base_path, + shard_id, + 2, + )); + let (epoch, _manifest) = manifest_store.claim_epoch(0).await.unwrap(); + + // Primary-key index on `id`, no user indexes. + let schema = create_pk_schema(); + let mut memtable = MemTable::new(schema.clone(), 1, vec![0]).unwrap(); + let mut registry = IndexStore::new(); + registry.enable_pk_index(&[("id".to_string(), 0)]); + memtable.set_indexes(registry); + + // id=1 updated in-gen (a -> a2); id=2 unique. + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 1])), + Arc::new(StringArray::from(vec!["a", "b", "a2"])), + ], + ) + .unwrap(); + let frag_id = memtable.insert(batch).await.unwrap(); + memtable.mark_wal_flushed(&[frag_id], 1, &[0]); + + let flusher = MemTableFlusher::new( + store.clone(), + base_path.clone(), + _base_uri.clone(), + shard_id, + manifest_store.clone(), + ); + let result = flusher + .flush_with_indexes(&memtable, epoch, &[], 1) + .await + .unwrap(); + + // Reopen the sidecar directly by path (the block-list's route). + let gen_path = base_path + .clone() + .join("_mem_wal") + .join(shard_id.to_string()) + .join(result.generation.path.as_str()); + let index_store = Arc::new(LanceIndexStore::new( + store.clone(), + pk_index_path(&gen_path), + Arc::new(LanceCache::no_cache()), + )); + let registry = IndexPluginRegistry::with_default_plugins(); + let plugin = registry.get_plugin_by_name("BTree").unwrap(); + let details = + prost_types::Any::from_msg(&lance_index::pbold::BTreeIndexDetails::default()).unwrap(); + let index = plugin + .load_index(index_store, &details, None, &LanceCache::no_cache()) + .await + .unwrap(); + + let contains = |id: i32| { + let index = index.clone(); + async move { + let result = index + .search( + &SargableQuery::Equals(ScalarValue::Int32(Some(id))), + &NoOpMetricsCollector, + ) + .await + .unwrap(); + match result { + SearchResult::Exact(s) | SearchResult::AtMost(s) | SearchResult::AtLeast(s) => { + !s.is_empty() + } + } + } + }; + // Both PKs present (id=1 even though its first version was superseded); + // an absent PK is not. + assert!(contains(1).await); + assert!(contains(2).await); + assert!(!contains(99).await); + } + /// Covers `finalize_generation` writing both a deletion vector *and* /// indexes into the same manifest — the deletion-only and index-only /// paths are exercised by sibling tests. diff --git a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs index 04c77b77e28..bd7253e3c7e 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs @@ -4,91 +4,121 @@ //! Per-source block-list construction for LSM vector search. //! //! A generation's membership is a [`GenMembership`]: in-memory generations -//! (active / frozen) are **probed by value** against their maintained -//! primary-key BTrees (no per-query set), while on-disk generations (flushed, -//! base) carry a cached `Arc>` of PK hashes ([`compute_pk_hash`]). -//! Each source gets a `Vec` of the newer generations -//! (`NEWER(G)`; base: all of them); the KNN drops a candidate whose PK is in any -//! (see [`super::exec::PkHashFilterExec`]). +//! (active / frozen) are probed by value against their maintained primary-key +//! index (no per-query set), while flushed generations are probed against their +//! standalone on-disk PK BTree (the sidecar written at flush, opened by path and +//! queried with `Equals`). Each source gets a `Vec` of the newer +//! generations (`NEWER(G)`; base: all of them); the KNN drops a candidate whose +//! PK any of them contains (see [`super::exec::PkBlockFilterExec`]). //! -//! Cross-generation only: within-gen dups share a hash and fall to the global -//! dedup's `(generation, freshness)` tiebreaker. +//! Cross-generation only: within-gen dups collapse via the global dedup's +//! `(generation, freshness)` tiebreaker. -use std::collections::{HashMap, HashSet}; -use std::sync::Arc; +use std::collections::HashMap; +use std::sync::{Arc, LazyLock}; -use arrow_array::RecordBatch; use datafusion::common::ScalarValue; -use futures::TryStreamExt; -use lance_core::Result; - +use lance_core::cache::LanceCache; +use lance_core::{Error, Result}; + +use lance_index::metrics::NoOpMetricsCollector; +use lance_index::registry::IndexPluginRegistry; +use lance_index::scalar::lance_format::LanceIndexStore; +use lance_index::scalar::{ + IndexStore as ScalarIndexStore, SargableQuery, ScalarIndex, SearchResult, +}; use uuid::Uuid; use super::data_source::{LsmDataSource, LsmGeneration}; -use super::exec::{compute_pk_hash, resolve_pk_indices}; use super::flushed_cache::{FlushedMemTableCache, open_flushed_dataset}; -use crate::dataset::Dataset; +use crate::dataset::mem_wal::index::encode_pk_tuple; +use crate::dataset::mem_wal::util::PK_INDEX_DIR; use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; use crate::session::Session; +/// Default-plugin registry, used only to load the standalone PK BTree by its +/// `BTreeIndexDetails` type. Built once. +static PK_BTREE_REGISTRY: LazyLock> = + LazyLock::new(IndexPluginRegistry::with_default_plugins); + /// One newer generation's PK membership, used to decide whether it shadows an -/// older source's row. In-memory generations probe their primary-key BTrees by -/// value (snapshot-bounded, so a not-yet-visible write can't shadow an older -/// visible copy); on-disk generations carry a cached PK-hash set. -#[derive(Debug, Clone)] +/// older source's row. +#[derive(Clone, Debug)] pub enum GenMembership { - /// Probe the in-memory memtable's index, bounded to its visible prefix. - Index { + /// Probe the in-memory memtable's primary-key index, bounded to its visible + /// prefix (so a not-yet-visible write can't shadow an older visible copy). + InMemory { index_store: Arc, /// Inclusive visible row watermark; `None` when no rows are visible. max_visible_row: Option, }, - /// A materialized PK-hash set (flushed / base, or an in-memory memtable that - /// has no primary-key index). - Set(Arc>), + /// Probe the flushed generation's standalone on-disk PK BTree. + OnDisk(Arc), } impl GenMembership { - /// Whether this generation visibly contains the primary key. The on-disk - /// `Set` case matches on `pk_hash`; the in-memory `Index` case probes by - /// `pk_values` (collision-free). The caller supplies both for a candidate - /// row — see [`Self::needs_values`]. - pub fn contains(&self, pk_hash: u64, pk_values: &[ScalarValue]) -> bool { + /// Whether this generation visibly contains the primary key. The caller + /// supplies both the PK `values` (for the in-memory probe) and the + /// pre-built `on_disk_key` (typed value for a single-column PK, encoded + /// `Binary` tuple for a composite one — see [`on_disk_pk_key`]). + pub async fn contains( + &self, + values: &[ScalarValue], + on_disk_key: &ScalarValue, + ) -> Result { match self { - Self::Index { + Self::InMemory { index_store, max_visible_row, } => { - let Some(max) = max_visible_row else { - return false; - }; - index_store.pk_contains_visible(pk_values, *max) + Ok(max_visible_row.is_some_and(|max| index_store.pk_contains_visible(values, max))) + } + Self::OnDisk(index) => { + let result = index + .search( + &SargableQuery::Equals(on_disk_key.clone()), + &NoOpMetricsCollector, + ) + .await + .map_err(|e| Error::io(e.to_string()))?; + Ok(!search_is_empty(&result)) } - Self::Set(set) => set.contains(&pk_hash), } } - /// Whether [`Self::contains`] needs the primary-key values (the `Index` - /// case) rather than just the hash. Lets the filter skip per-row value - /// extraction when every membership is an on-disk `Set`. - pub fn needs_values(&self) -> bool { - matches!(self, Self::Index { .. }) - } - /// Whether this generation has no (visible) membership — used to skip adding - /// an empty blocked set. Approximate for the index case (it ignores the - /// watermark when counting), which only ever leaves a harmless no-op entry. + /// an empty blocked set. A flushed generation always has rows (flush rejects + /// an empty memtable), so it is never empty. fn is_empty(&self) -> bool { match self { - Self::Index { + Self::InMemory { index_store, max_visible_row, } => max_visible_row.is_none() || index_store.pk_is_empty(), - Self::Set(set) => set.is_empty(), + Self::OnDisk(_) => false, } } } +/// Whether a scalar search returned no rows (existence test for the block-list). +fn search_is_empty(result: &SearchResult) -> bool { + match result { + SearchResult::Exact(set) | SearchResult::AtMost(set) | SearchResult::AtLeast(set) => { + set.is_empty() + } + } +} + +/// The probe key for the on-disk PK BTree: a single-column PK indexes its typed +/// value directly; a composite PK indexes the order-preserving encoded tuple as +/// `Binary` (matching what flush wrote — see [`encode_pk_tuple`]). +pub fn on_disk_pk_key(values: &[ScalarValue]) -> Result { + match values { + [single] => Ok(single.clone()), + _ => Ok(ScalarValue::Binary(Some(encode_pk_tuple(values)?))), + } +} + /// Per-source blocked memberships, keyed by `(shard_id, generation)`. Each value /// is the memberships of the generations newer than that source. pub type SourceBlockLists = HashMap<(Option, LsmGeneration), Vec>; @@ -103,11 +133,10 @@ type ShardGenSets = HashMap>; /// Only superseded sources get an entry; the newest of each shard never does. pub async fn compute_source_block_lists( sources: &[LsmDataSource], - pk_columns: &[String], session: Option<&Arc>, flushed_cache: Option<&Arc>, ) -> Result { - // Hash each non-base source's membership, grouped by shard (generations are + // Membership per non-base source, grouped by shard (generations are // per-shard, so supersession is within-shard only). let mut by_shard: ShardGenSets = HashMap::new(); let mut has_base = false; @@ -121,7 +150,7 @@ pub async fn compute_source_block_lists( generation, .. } => { - let membership = in_memory_membership(batch_store, index_store, pk_columns)?; + let membership = in_memory_membership(batch_store, index_store); by_shard .entry(*shard_id) .or_default() @@ -133,12 +162,11 @@ pub async fn compute_source_block_lists( generation, .. } => { - // Cached by immutable path so repeated searches skip the PK scan. - let hashes = flushed_pk_hashes(path, pk_columns, session, flushed_cache).await?; + let index = open_pk_index(path, session, flushed_cache).await?; by_shard .entry(*shard_id) .or_default() - .push((*generation, GenMembership::Set(hashes))); + .push((*generation, GenMembership::OnDisk(index))); } } } @@ -168,13 +196,12 @@ pub async fn compute_source_block_lists( /// The fresh-tier block-list: one [`GenMembership`] per generation that shadows /// the base table — active + frozen memtables (probed against their index) and -/// flushed generations (cached PK-hash sets). A base/external reader can test -/// any PK against these (e.g. via `contains`) to decide whether the fresh tier -/// shadows it. The base source, if present, is skipped (it is what gets -/// shadowed). +/// flushed generations (probed against their on-disk PK BTree). A base/external +/// reader can test any PK against these (via [`GenMembership::contains`]) to +/// decide whether the fresh tier shadows it. The base source, if present, is +/// skipped (it is what gets shadowed). pub async fn fresh_tier_block_list( sources: &[LsmDataSource], - pk_columns: &[String], session: Option<&Arc>, flushed_cache: Option<&Arc>, ) -> Result> { @@ -186,10 +213,10 @@ pub async fn fresh_tier_block_list( batch_store, index_store, .. - } => in_memory_membership(batch_store, index_store, pk_columns)?, - LsmDataSource::FlushedMemTable { path, .. } => GenMembership::Set( - flushed_pk_hashes(path, pk_columns, session, flushed_cache).await?, - ), + } => in_memory_membership(batch_store, index_store), + LsmDataSource::FlushedMemTable { path, .. } => { + GenMembership::OnDisk(open_pk_index(path, session, flushed_cache).await?) + } }; if !membership.is_empty() { memberships.push(membership); @@ -198,260 +225,234 @@ pub async fn fresh_tier_block_list( Ok(memberships) } -/// Cross-source membership of an in-memory (active / frozen) memtable. -/// -/// Prefers a snapshot-bounded **probe** of the maintained primary-key index (no -/// per-query set built), falling back to a one-time `BatchStore` scan only when -/// the memtable has no such index (e.g. a table without a primary key) — which -/// the production vector-search path never hits, since that index is always -/// enabled alongside the secondary indexes. +/// Cross-source membership of an in-memory (active / frozen) memtable: a +/// snapshot-bounded probe of its maintained primary-key index. A memtable +/// without a primary-key index can't be probed, so it blocks nothing — the +/// production vector-search path always enables the index. fn in_memory_membership( batch_store: &Arc, index_store: &Arc, - pk_columns: &[String], -) -> Result { - if index_store.has_pk_index() { - let max_visible_row = batch_store.max_visible_row(index_store.max_visible_batch_position()); - Ok(GenMembership::Index { - index_store: index_store.clone(), - max_visible_row, - }) - } else { - Ok(GenMembership::Set(Arc::new(pk_hashes_from_batch_store( - batch_store, - pk_columns, - )?))) +) -> GenMembership { + let max_visible_row = batch_store.max_visible_row(index_store.max_visible_batch_position()); + GenMembership::InMemory { + index_store: index_store.clone(), + max_visible_row, } } -/// Hash the PK membership of an in-memory memtable (active or frozen) from its -/// committed `BatchStore` rows. -pub fn pk_hashes_from_batch_store( - store: &BatchStore, - pk_columns: &[String], -) -> Result> { - let mut batches: Vec = Vec::with_capacity(store.len()); - for i in 0..store.len() { - if let Some(stored) = store.get(i) { - batches.push(stored.data.clone()); - } - } - pk_hashes_from_batches(&batches, pk_columns) -} - -/// Hash every row's primary key across `batches` into a membership set. -fn pk_hashes_from_batches(batches: &[RecordBatch], pk_columns: &[String]) -> Result> { - let mut pk_hashes = HashSet::new(); - for batch in batches { - if batch.num_rows() == 0 { - continue; - } - let pk_indices = resolve_pk_indices(batch, pk_columns) - .map_err(|e| lance_core::Error::invalid_input(e.to_string()))?; - for row_idx in 0..batch.num_rows() { - pk_hashes.insert(compute_pk_hash(batch, &pk_indices, row_idx)); - } - } - Ok(pk_hashes) -} - -/// Build (or fetch the cached) PK-hash membership for one flushed generation. -/// Cached by immutable path (single-flight); the build scans the flushed -/// dataset's PK columns. -async fn flushed_pk_hashes( +/// Open (or fetch the cached) standalone PK index for one flushed generation. +/// Cached by immutable path (single-flight); pages go through a shared cache. +async fn open_pk_index( path: &str, - pk_columns: &[String], session: Option<&Arc>, flushed_cache: Option<&Arc>, -) -> Result>> { +) -> Result> { match flushed_cache { Some(cache) => { - let build_cache = cache.clone(); + let page_cache = cache.index_page_cache(); let build_path = path.to_string(); let build_session = session.cloned(); - let build_pk = pk_columns.to_vec(); + let build_cache = cache.clone(); cache - .get_or_build_pk_hashes( + .get_or_open_pk_index( path, - // `Box::pin` keeps this build future off the caller's future + // `Box::pin` keeps this open future off the caller's future // (avoids `clippy::large_futures`). Box::pin(async move { - let dataset = open_flushed_dataset( + build_pk_index( &build_path, build_session.as_ref(), Some(&build_cache), + page_cache, ) - .await?; - scan_pk_hashes(&dataset, &build_pk).await + .await }), ) .await } - None => { - let dataset = open_flushed_dataset(path, session, None).await?; - Ok(Arc::new(scan_pk_hashes(&dataset, pk_columns).await?)) - } + None => build_pk_index(path, session, None, Arc::new(LanceCache::no_cache())).await, } } -/// Scan a dataset's PK columns and fold them into a membership set, one batch -/// resident at a time (no full PK-column buffer). -async fn scan_pk_hashes(dataset: &Dataset, pk_columns: &[String]) -> Result> { - let pk_refs: Vec<&str> = pk_columns.iter().map(String::as_str).collect(); - let mut scanner = dataset.scan(); - scanner.project(&pk_refs)?; - let mut stream = scanner.try_into_stream().await?; - let mut hashes = HashSet::new(); - while let Some(batch) = stream.try_next().await? { - if batch.num_rows() == 0 { - continue; - } - let pk_indices = resolve_pk_indices(&batch, pk_columns) - .map_err(|e| lance_core::Error::invalid_input(e.to_string()))?; - for row in 0..batch.num_rows() { - hashes.insert(compute_pk_hash(&batch, &pk_indices, row)); - } +/// Open the standalone PK BTree at `{flushed gen}/_pk_index`. Reuses the flushed +/// dataset's (session-configured) object store, then loads the sidecar index +/// directly by path through the BTree plugin — it is not a manifest index. +async fn build_pk_index( + path: &str, + session: Option<&Arc>, + flushed_cache: Option<&Arc>, + page_cache: Arc, +) -> Result> { + let dataset = open_flushed_dataset(path, session, flushed_cache).await?; + let index_dir = dataset.base.clone().join(PK_INDEX_DIR); + let store: Arc = Arc::new(LanceIndexStore::new( + dataset.object_store.clone(), + index_dir, + page_cache.clone(), + )); + let plugin = PK_BTREE_REGISTRY.get_plugin_by_name("BTree")?; + let details = prost_types::Any::from_msg(&lance_index::pbold::BTreeIndexDetails::default()) + .map_err(|e| Error::io(e.to_string()))?; + plugin + .load_index(store, &details, None, page_cache.as_ref()) + .await +} + +/// Test helper: write a flushed generation's standalone PK sidecar at +/// `{uri}/_pk_index` from `batches`, mirroring what flush does in production. +/// `pk_columns` are the primary-key column names (field ids are synthesized by +/// position — `insert` resolves columns by name). A no-op when no batch carries +/// the PK columns. +#[cfg(test)] +pub async fn write_test_pk_sidecar( + uri: &str, + batches: &[arrow_array::RecordBatch], + pk_columns: &[&str], +) -> Result<()> { + use datafusion::physical_plan::stream::RecordBatchStreamAdapter; + use lance_index::scalar::btree::train_btree_index; + use lance_io::object_store::ObjectStore; + + use crate::dataset::mem_wal::util::pk_index_path; + + let pk: Vec<(String, i32)> = pk_columns + .iter() + .enumerate() + .map(|(i, c)| (c.to_string(), i as i32)) + .collect(); + let mut index = IndexStore::new(); + index.enable_pk_index(&pk); + let mut offset = 0u64; + for batch in batches { + index.insert(batch, offset)?; + offset += batch.num_rows() as u64; } - Ok(hashes) + + let training = index.pk_training_batches(8192)?; + if training.is_empty() { + return Ok(()); + } + let schema = training[0].schema(); + let (object_store, base_path) = ObjectStore::from_uri(uri).await?; + let store = LanceIndexStore::new( + object_store, + pk_index_path(&base_path), + Arc::new(LanceCache::no_cache()), + ); + let stream = Box::pin(RecordBatchStreamAdapter::new( + schema, + futures::stream::iter(training.into_iter().map(Ok)), + )); + train_btree_index(stream, &store, 8192, None, None).await } #[cfg(test)] mod tests { use super::*; - use arrow_array::Int32Array; + use crate::dataset::mem_wal::scanner::data_source::{LsmDataSource, LsmGeneration}; + use crate::dataset::mem_wal::write::IndexStore; + use arrow_array::{Int32Array, RecordBatch}; use arrow_schema::{DataType, Field, Schema}; use std::sync::Arc; + use uuid::Uuid; fn id_batch(ids: &[i32]) -> RecordBatch { let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(ids.to_vec()))]).unwrap() } - /// Hash a single Int32 `id` PK the way the planner does, so a test can probe - /// a returned blocked set by value. - fn hash_id(id: i32) -> u64 { - let batch = id_batch(&[id]); - let pk_indices = resolve_pk_indices(&batch, &["id".to_string()]).unwrap(); - compute_pk_hash(&batch, &pk_indices, 0) + /// An active/frozen memtable source whose PK index holds one row per id in + /// `ids` (positions 0..n), all committed and visible. + fn active_source(shard: Uuid, generation: u64, ids: &[i32]) -> LsmDataSource { + let store = BatchStore::with_capacity(16); + let mut index = IndexStore::new(); + index.enable_pk_index(&[("id".to_string(), 0)]); + for &id in ids { + let b = id_batch(&[id]); + let (bp, off, _) = store.append(b.clone()).unwrap(); + index.insert_with_batch_position(&b, off, Some(bp)).unwrap(); + } + LsmDataSource::ActiveMemTable { + batch_store: Arc::new(store), + index_store: Arc::new(index), + schema: id_batch(&[1]).schema(), + shard_id: shard, + generation: LsmGeneration::memtable(generation), + } } - /// Whether `id`'s PK is blocked by any of a source's newer-gen memberships - /// (supplying both the hash and the value, as the filter does). - fn blocks(memberships: &[GenMembership], id: i32) -> bool { + /// Whether `id`'s PK is blocked by any of a source's newer-gen memberships. + async fn blocks(memberships: &[GenMembership], id: i32) -> bool { let values = [ScalarValue::Int32(Some(id))]; - memberships.iter().any(|m| m.contains(hash_id(id), &values)) - } - - #[test] - fn pk_hashes_collapse_within_gen_duplicates() { - // Two rows share pk=1 (a within-gen duplicate); pk=2 is unique. - let hashes = pk_hashes_from_batches(&[id_batch(&[1, 2, 1])], &["id".to_string()]).unwrap(); - assert_eq!(hashes.len(), 2); // distinct pks: 1, 2 - } - - #[test] - fn empty_batches_yield_empty_membership() { - let hashes = pk_hashes_from_batches(&[id_batch(&[])], &["id".to_string()]).unwrap(); - assert!(hashes.is_empty()); + let key = on_disk_pk_key(&values).unwrap(); + for m in memberships { + if m.contains(&values, &key).await.unwrap() { + return true; + } + } + false } #[test] - fn batch_store_membership_collapses_within_gen_dups() { - let store = BatchStore::with_capacity(8); - // Two single-row batches, both pk=1 (a within-gen update). - store.append(id_batch(&[1])).unwrap(); - store.append(id_batch(&[1])).unwrap(); - // A two-row batch: pk=2, pk=3. - store.append(id_batch(&[2, 3])).unwrap(); - - let hashes = pk_hashes_from_batch_store(&store, &["id".to_string()]).unwrap(); - assert_eq!(hashes.len(), 3); // distinct pks: 1, 2, 3 + fn on_disk_key_is_typed_for_single_and_binary_for_composite() { + // Single-column → the typed value; composite → encoded Binary. + let single = [ScalarValue::Int32(Some(7))]; + assert_eq!( + on_disk_pk_key(&single).unwrap(), + ScalarValue::Int32(Some(7)) + ); + let composite = [ScalarValue::Int32(Some(1)), ScalarValue::from("a")]; + assert!(matches!( + on_disk_pk_key(&composite).unwrap(), + ScalarValue::Binary(Some(_)) + )); } #[tokio::test] - async fn fresh_tier_block_list_one_set_per_in_memory_gen() { - use crate::dataset::mem_wal::scanner::data_source::{LsmDataSource, LsmGeneration}; - use crate::dataset::mem_wal::write::IndexStore; - use uuid::Uuid; - + async fn fresh_tier_block_list_one_membership_per_in_memory_gen() { let shard = Uuid::new_v4(); - let mk = |ids: &[i32], generation: u64| { - let store = BatchStore::with_capacity(8); - store.append(id_batch(ids)).unwrap(); - LsmDataSource::ActiveMemTable { - batch_store: Arc::new(store), - index_store: Arc::new(IndexStore::new()), - schema: id_batch(&[1]).schema(), - shard_id: shard, - generation: LsmGeneration::memtable(generation), - } - }; // Active gen 2: pk=1,2. Frozen gen 1: pk=3. - let sources = vec![mk(&[1, 2], 2), mk(&[3], 1)]; + let sources = vec![ + active_source(shard, 2, &[1, 2]), + active_source(shard, 1, &[3]), + ]; - let memberships = fresh_tier_block_list(&sources, &["id".to_string()], None, None) - .await - .unwrap(); + let memberships = fresh_tier_block_list(&sources, None, None).await.unwrap(); // One membership per generation; together they cover pk=1,2,3 (not 4). assert_eq!(memberships.len(), 2); - let fresh_blocks = |id: i32| blocks(&memberships, id); for id in [1, 2, 3] { - assert!(fresh_blocks(id)); + assert!(blocks(&memberships, id).await); } - assert!(!fresh_blocks(4)); + assert!(!blocks(&memberships, 4).await); } #[tokio::test] async fn block_lists_suppress_stale_across_in_memory_gens() { - use crate::dataset::mem_wal::scanner::data_source::{LsmDataSource, LsmGeneration}; - use crate::dataset::mem_wal::write::IndexStore; - use uuid::Uuid; - let shard = Uuid::new_v4(); - let mk = |batches: &[&[i32]], generation: u64| { - let store = BatchStore::with_capacity(8); - for ids in batches { - store.append(id_batch(ids)).unwrap(); - } - LsmDataSource::ActiveMemTable { - batch_store: Arc::new(store), - index_store: Arc::new(IndexStore::new()), - schema: id_batch(&[1]).schema(), - shard_id: shard, - generation: LsmGeneration::memtable(generation), - } - }; - - // Frozen gen 1: stale pk=1. - // Active gen 2: pk=1 re-written, pk=2 new. - let sources = vec![mk(&[&[1]], 1), mk(&[&[1], &[2]], 2)]; + // Frozen gen 1: stale pk=1. Active gen 2: pk=1 re-written, pk=2 new. + let sources = vec![ + active_source(shard, 1, &[1]), + active_source(shard, 2, &[1, 2]), + ]; - let blocked = Box::pin(compute_source_block_lists( - &sources, - &["id".to_string()], - None, - None, - )) - .await - .unwrap(); + let blocked = Box::pin(compute_source_block_lists(&sources, None, None)) + .await + .unwrap(); let g1 = LsmGeneration::memtable(1); let g2 = LsmGeneration::memtable(2); // The newer active write supersedes the frozen copy: gen 1 is blocked on // pk=1, so its KNN drops pk=1. - assert!(blocks(&blocked[&(Some(shard), g1)], 1)); + assert!(blocks(&blocked[&(Some(shard), g1)], 1).await); // The active (newest) generation is superseded by nothing — no entry. assert!(!blocked.contains_key(&(Some(shard), g2))); } #[tokio::test] async fn block_lists_suppress_stale_base_row() { - use crate::dataset::mem_wal::scanner::data_source::{LsmDataSource, LsmGeneration}; - use crate::dataset::mem_wal::write::IndexStore; use crate::dataset::{Dataset, WriteParams}; use arrow_array::RecordBatchIterator; - use uuid::Uuid; // Base (gen 0): pk=1 (stale), pk=3 (live). let base_batch = id_batch(&[1, 3]); @@ -466,174 +467,65 @@ mod tests { ); // Active gen 1: pk=1 re-written, pk=2 new. - let store = BatchStore::with_capacity(8); - store.append(id_batch(&[1])).unwrap(); - store.append(id_batch(&[2])).unwrap(); - let sources = vec![ LsmDataSource::BaseTable { dataset: base }, - LsmDataSource::ActiveMemTable { - batch_store: Arc::new(store), - index_store: Arc::new(IndexStore::new()), - schema, - shard_id: Uuid::new_v4(), - generation: LsmGeneration::memtable(1), - }, + active_source(Uuid::new_v4(), 1, &[1, 2]), ]; - let blocked = Box::pin(compute_source_block_lists( - &sources, - &["id".to_string()], - None, - None, - )) - .await - .unwrap(); + let blocked = Box::pin(compute_source_block_lists(&sources, None, None)) + .await + .unwrap(); // Base is blocked by every newer gen: pk=1 (re-written in gen 1) is - // blocked, pk=3 (base-only) is not. End-to-end drop: vector_search specs. + // blocked, pk=3 (base-only) is not. let base_blocked = blocked .get(&(None, LsmGeneration::BASE_TABLE)) .expect("base has a blocked set"); - assert!(blocks(base_blocked, 1)); - assert!(!blocks(base_blocked, 3)); + assert!(blocks(base_blocked, 1).await); + assert!(!blocks(base_blocked, 3).await); } #[tokio::test] async fn block_lists_are_keyed_per_shard() { // Regression: generations are per-shard, so a source must only be blocked - // by newer generations of its OWN shard. A generation-only key would - // cross-block same-generation sources from different shards. - use crate::dataset::mem_wal::scanner::data_source::{LsmDataSource, LsmGeneration}; - use crate::dataset::mem_wal::write::IndexStore; - use uuid::Uuid; - - let mk = |shard: Uuid, ids: &[i32], generation: u64| { - let store = BatchStore::with_capacity(8); - store.append(id_batch(ids)).unwrap(); - LsmDataSource::ActiveMemTable { - batch_store: Arc::new(store), - index_store: Arc::new(IndexStore::new()), - schema: id_batch(&[1]).schema(), - shard_id: shard, - generation: LsmGeneration::memtable(generation), - } - }; - - // Two shards, each: frozen gen 1 (stale) + active gen 2 (re-write). - // Shard A keys pk=1; shard B keys pk=2 (disjoint partitions). + // by newer generations of its OWN shard. let a = Uuid::new_v4(); let b = Uuid::new_v4(); + // Two shards, each: frozen gen 1 (stale) + active gen 2 (re-write). + // Shard A keys pk=1; shard B keys pk=2 (disjoint partitions). let sources = vec![ - mk(a, &[1], 1), - mk(a, &[1], 2), - mk(b, &[2], 1), - mk(b, &[2], 2), + active_source(a, 1, &[1]), + active_source(a, 2, &[1]), + active_source(b, 1, &[2]), + active_source(b, 2, &[2]), ]; - let blocked = Box::pin(compute_source_block_lists( - &sources, - &["id".to_string()], - None, - None, - )) - .await - .unwrap(); + let blocked = Box::pin(compute_source_block_lists(&sources, None, None)) + .await + .unwrap(); let g1 = LsmGeneration::memtable(1); let g2 = LsmGeneration::memtable(2); // Each shard's gen 1 is blocked by its OWN gen 2 only. - assert!(blocks(&blocked[&(Some(a), g1)], 1)); - assert!(!blocks(&blocked[&(Some(a), g1)], 2)); - assert!(blocks(&blocked[&(Some(b), g1)], 2)); - assert!(!blocks(&blocked[&(Some(b), g1)], 1)); + assert!(blocks(&blocked[&(Some(a), g1)], 1).await); + assert!(!blocks(&blocked[&(Some(a), g1)], 2).await); + assert!(blocks(&blocked[&(Some(b), g1)], 2).await); + assert!(!blocks(&blocked[&(Some(b), g1)], 1).await); // The newest generation of each shard is superseded by nothing. assert!(!blocked.contains_key(&(Some(a), g2))); assert!(!blocked.contains_key(&(Some(b), g2))); } - #[tokio::test] - async fn in_memory_membership_reads_from_pk_index() { - // When the memtable has a maintained PK-position index, the block-list - // sources its membership from that index (no BatchStore re-scan) and - // still suppresses an older generation's stale copy. - use crate::dataset::mem_wal::scanner::data_source::{LsmDataSource, LsmGeneration}; - use crate::dataset::mem_wal::write::IndexStore; - use uuid::Uuid; - - let shard = Uuid::new_v4(); - - // Frozen gen 1: stale pk=1, no PK-position index (exercises the fallback). - let stale_store = BatchStore::with_capacity(8); - stale_store.append(id_batch(&[1])).unwrap(); - - // Active gen 2: pk=1 re-written + pk=2, with the index enabled + populated. - // `insert_with_batch_position(Some(bp))` advances the visibility watermark - // so the snapshot-bounded probe sees both rows. - let active_store = BatchStore::with_capacity(8); - let mut active_index = IndexStore::new(); - active_index.enable_pk_index(&[("id".to_string(), 0)]); - let b1 = id_batch(&[1]); - let (bp1, off1, _) = active_store.append(b1.clone()).unwrap(); - active_index - .insert_with_batch_position(&b1, off1, Some(bp1)) - .unwrap(); - let b2 = id_batch(&[2]); - let (bp2, off2, _) = active_store.append(b2.clone()).unwrap(); - active_index - .insert_with_batch_position(&b2, off2, Some(bp2)) - .unwrap(); - - let schema = id_batch(&[1]).schema(); - let sources = vec![ - LsmDataSource::ActiveMemTable { - batch_store: Arc::new(stale_store), - index_store: Arc::new(IndexStore::new()), - schema: schema.clone(), - shard_id: shard, - generation: LsmGeneration::memtable(1), - }, - LsmDataSource::ActiveMemTable { - batch_store: Arc::new(active_store), - index_store: Arc::new(active_index), - schema, - shard_id: shard, - generation: LsmGeneration::memtable(2), - }, - ]; - - let blocked = Box::pin(compute_source_block_lists( - &sources, - &["id".to_string()], - None, - None, - )) - .await - .unwrap(); - - // gen 2's index-sourced membership is {pk=1, pk=2}; gen 1 (stale pk=1) - // is blocked on both, and the newest gen 2 has no blocked set. - let g1 = LsmGeneration::memtable(1); - assert!(blocks(&blocked[&(Some(shard), g1)], 1)); - assert!(blocks(&blocked[&(Some(shard), g1)], 2)); - assert!(!blocked.contains_key(&(Some(shard), LsmGeneration::memtable(2)))); - } - #[tokio::test] async fn index_membership_is_snapshot_bounded() { // The index-sourced membership only counts a PK whose version is visible // at the source's watermark, so a newer generation's not-yet-visible // write can't shadow an older generation's visible copy. - use crate::dataset::mem_wal::scanner::data_source::{LsmDataSource, LsmGeneration}; - use crate::dataset::mem_wal::write::IndexStore; - use uuid::Uuid; - let shard = Uuid::new_v4(); let schema = id_batch(&[1]).schema(); - // Older frozen gen 1: pk=1 (no PK-position index → Set fallback). - let g1_store = BatchStore::with_capacity(8); - g1_store.append(id_batch(&[1])).unwrap(); + // Older frozen gen 1: pk=1. + let g1 = active_source(shard, 1, &[1]); // Newer active gen 2: pk=99 visible at position 0, then pk=1 written at // position 1 but with the watermark left at batch 0 (so pk=1 is in the @@ -651,39 +543,24 @@ mod tests { g2_index .insert_with_batch_position(&b1, off1, None) // index updated, watermark unchanged .unwrap(); + let g2 = LsmDataSource::ActiveMemTable { + batch_store: Arc::new(g2_store), + index_store: Arc::new(g2_index), + schema, + shard_id: shard, + generation: LsmGeneration::memtable(2), + }; - let sources = vec![ - LsmDataSource::ActiveMemTable { - batch_store: Arc::new(g1_store), - index_store: Arc::new(IndexStore::new()), - schema: schema.clone(), - shard_id: shard, - generation: LsmGeneration::memtable(1), - }, - LsmDataSource::ActiveMemTable { - batch_store: Arc::new(g2_store), - index_store: Arc::new(g2_index), - schema, - shard_id: shard, - generation: LsmGeneration::memtable(2), - }, - ]; - - let blocked = Box::pin(compute_source_block_lists( - &sources, - &["id".to_string()], - None, - None, - )) - .await - .unwrap(); + let blocked = Box::pin(compute_source_block_lists(&[g1, g2], None, None)) + .await + .unwrap(); let g1_block = &blocked[&(Some(shard), LsmGeneration::memtable(1))]; // pk=99 is visible in gen 2 → it blocks gen 1's pk=99. - assert!(blocks(g1_block, 99)); + assert!(blocks(g1_block, 99).await); // pk=1's only gen-2 copy is not yet visible → it must NOT shadow gen 1. assert!( - !blocks(g1_block, 1), + !blocks(g1_block, 1).await, "a not-yet-visible newer write must not shadow an older visible copy" ); } diff --git a/rust/lance/src/dataset/mem_wal/scanner/builder.rs b/rust/lance/src/dataset/mem_wal/scanner/builder.rs index 1f6df3d1b32..42db4d9694f 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/builder.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/builder.rs @@ -459,31 +459,30 @@ impl LsmScanner { let sources = self.build_collector().collect()?; let memberships = super::block_list::fresh_tier_block_list( &sources, - &self.pk_columns, self.session.as_ref(), self.flushed_cache.as_ref(), ) .await?; let pk_indices = super::exec::resolve_pk_indices(pks, &self.pk_columns) .map_err(|e| Error::invalid_input(e.to_string()))?; - // On-disk generations probe by hash; in-memory ones probe their - // primary-key BTrees by value. Extract values only when needed. - let needs_values = memberships - .iter() - .any(super::block_list::GenMembership::needs_values); let mut contained = Vec::with_capacity(pks.num_rows()); for row in 0..pks.num_rows() { - let hash = super::exec::compute_pk_hash(pks, &pk_indices, row); - let values: Vec = if needs_values { - pk_indices - .iter() - .map(|&col| ScalarValue::try_from_array(pks.column(col), row)) - .collect::>() - .map_err(|e| Error::invalid_input(e.to_string()))? - } else { - Vec::new() - }; - contained.push(memberships.iter().any(|m| m.contains(hash, &values))); + // In-memory generations probe by value; flushed ones probe their + // on-disk PK BTree with the typed/encoded key. + let values: Vec = pk_indices + .iter() + .map(|&col| ScalarValue::try_from_array(pks.column(col), row)) + .collect::>() + .map_err(|e| Error::invalid_input(e.to_string()))?; + let on_disk_key = super::block_list::on_disk_pk_key(&values)?; + let mut found = false; + for membership in &memberships { + if membership.contains(&values, &on_disk_key).await? { + found = true; + break; + } + } + contained.push(found); } Ok(contained) } @@ -602,10 +601,14 @@ mod tests { }; let mk = |ids: &[i32], generation: u64| { let store = BatchStore::with_capacity(8); - store.append(id_batch(ids)).unwrap(); + let mut index = IndexStore::new(); + index.enable_pk_index(&[("id".to_string(), 0)]); + let b = id_batch(ids); + let (bp, off, _) = store.append(b.clone()).unwrap(); + index.insert_with_batch_position(&b, off, Some(bp)).unwrap(); InMemoryMemTableRef { batch_store: Arc::new(store), - index_store: Arc::new(IndexStore::new()), + index_store: Arc::new(index), schema: schema.clone(), generation, } diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec.rs b/rust/lance/src/dataset/mem_wal/scanner/exec.rs index 4603f96b532..115cffccc81 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec.rs @@ -9,7 +9,7 @@ //! - [`MemtableGenTagExec`]: Wraps a scan to add `_memtable_gen` column //! - [`BloomFilterGuardExec`]: Guards child execution with bloom filter check //! - [`CoalesceFirstExec`]: Returns first non-empty result with short-circuit -//! - [`PkHashFilterExec`]: Drops rows whose PK hash was superseded by a newer generation (the cross-generation block-list) +//! - [`PkBlockFilterExec`]: Drops rows whose PK was superseded by a newer generation (the cross-generation block-list) //! - [`NewestPkFilterExec`]: Drops active-memtable hits that aren't the newest visible version of their PK (the within-source recency filter) mod bloom_guard; @@ -17,7 +17,7 @@ mod coalesce_first; mod generation_tag; mod newest_pk_filter; mod pk; -mod pk_hash_filter; +mod pk_block_filter; pub use bloom_guard::{BloomFilterGuardExec, compute_pk_hash_from_scalars}; pub use coalesce_first::CoalesceFirstExec; @@ -27,4 +27,4 @@ pub use pk::{ ROW_ADDRESS_COLUMN, compute_pk_hash, is_supported_pk_type, resolve_pk_indices, validate_pk_types, }; -pub use pk_hash_filter::PkHashFilterExec; +pub use pk_block_filter::PkBlockFilterExec; diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/pk.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/pk.rs index 94e8f0dabc4..0707eb5e8dd 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec/pk.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec/pk.rs @@ -4,7 +4,7 @@ //! Shared primary-key helpers for the LSM scanner execution nodes. //! //! Centralizes PK column resolution and per-row hashing so that every -//! consumer (e.g. [`super::PkHashFilterExec`], [`super::NewestPkFilterExec`]) +//! consumer (e.g. [`super::PkBlockFilterExec`], [`super::NewestPkFilterExec`]) //! resolves and hashes a primary key the same way. The row hash is kept //! consistent with the variants supported by [`super::compute_pk_hash_from_scalars`] //! so a single PK produces the same hash regardless of which exec consumes it. diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/pk_block_filter.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/pk_block_filter.rs new file mode 100644 index 00000000000..a3c4810d648 --- /dev/null +++ b/rust/lance/src/dataset/mem_wal/scanner/exec/pk_block_filter.rs @@ -0,0 +1,348 @@ +// SPDX-License-Identifier: Apache-2.0 +// SPDX-FileCopyrightText: Copyright The Lance Authors + +//! Drop superseded rows from a per-source result by primary-key membership. +//! +//! Drops a row when any newer generation's membership ([`GenMembership`]) +//! contains its primary key — in-memory generations probe their PK index by +//! value, flushed generations probe their on-disk PK BTree with `Equals`. Used +//! both as the KNN post-filter (vector search, with over-fetch) and the +//! cross-generation scan filter (`k = 0`). +//! +//! Cross-generation only: within-gen duplicates collapse via the global dedup's +//! `(generation, freshness)` tiebreaker. +//! +//! Post-filters an over-fetched KNN (the planner's `overfetch_factor`); warns +//! when a source had >= k candidates but < k survived (over-fetch too small). +//! +//! Perf note: the on-disk probe is one `Equals` per row per flushed generation. +//! It is not disk-bound in steady state — the opened index and its (small, +//! memtable-sized) pages are held by the injected `FlushedMemTableCache` / +//! `LanceCache`, so after the first touch every probe is memory-resident. The +//! residual per-row cost is the in-memory BTree search plus a `SearchResult` +//! allocation per lookup; a batched/existence-only membership probe (avoiding +//! the per-row allocation) is the future optimization. + +use std::any::Any; +use std::fmt; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use arrow::compute::filter_record_batch; +use arrow_array::{BooleanArray, RecordBatch}; +use arrow_schema::SchemaRef; +use datafusion::common::ScalarValue; +use datafusion::error::{DataFusionError, Result as DFResult}; +use datafusion::execution::TaskContext; +use datafusion::physical_expr::EquivalenceProperties; +use datafusion::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties, + SendableRecordBatchStream, +}; +use futures::future::BoxFuture; +use futures::{FutureExt, Stream, StreamExt}; +use tracing::warn; + +use super::super::block_list::{GenMembership, on_disk_pk_key}; +use super::pk::resolve_pk_indices; + +/// Filters out rows whose PK is contained in any newer generation's membership. +#[derive(Debug)] +pub struct PkBlockFilterExec { + input: Arc, + pk_columns: Vec, + /// Newer generations' membership; a row is blocked if any contains its PK. + blocked: Vec, + /// Target neighbor count, used only to warn on a per-source under-fetch. + k: usize, + properties: Arc, +} + +impl PkBlockFilterExec { + pub fn new( + input: Arc, + pk_columns: Vec, + blocked: Vec, + k: usize, + ) -> Self { + // A filter preserves the input schema and partitioning. + let properties = Arc::new(PlanProperties::new( + EquivalenceProperties::new(input.schema()), + input.output_partitioning().clone(), + input.pipeline_behavior(), + input.boundedness(), + )); + Self { + input, + pk_columns, + blocked, + k, + properties, + } + } +} + +impl DisplayAs for PkBlockFilterExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default + | DisplayFormatType::Verbose + | DisplayFormatType::TreeRender => { + write!( + f, + "PkBlockFilterExec: pk_cols=[{}], gens={}", + self.pk_columns.join(", "), + self.blocked.len(), + ) + } + } + } +} + +impl ExecutionPlan for PkBlockFilterExec { + fn name(&self) -> &str { + "PkBlockFilterExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.input.schema() + } + + fn properties(&self) -> &Arc { + &self.properties + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> DFResult> { + if children.len() != 1 { + return Err(DataFusionError::Internal( + "PkBlockFilterExec requires exactly one child".to_string(), + )); + } + Ok(Arc::new(Self::new( + children[0].clone(), + self.pk_columns.clone(), + self.blocked.clone(), + self.k, + ))) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> DFResult { + let input_stream = self.input.execute(partition, context)?; + Ok(Box::pin(PkBlockFilterStream { + input: input_stream, + pk_columns: self.pk_columns.clone(), + blocked: self.blocked.clone(), + k: self.k, + schema: self.schema(), + pending: None, + input_seen: 0, + kept: 0, + warned: false, + })) + } +} + +struct PkBlockFilterStream { + input: SendableRecordBatchStream, + pk_columns: Vec, + blocked: Vec, + k: usize, + schema: SchemaRef, + /// The in-flight filter for the batch currently being processed (the probe + /// is async, so a batch is filtered off-poll and resumed here). + pending: Option>>, + input_seen: usize, + kept: usize, + warned: bool, +} + +/// Keep only the rows no newer-gen membership contains. Async because flushed +/// generations are probed against their on-disk PK BTree. +async fn filter_batch( + batch: RecordBatch, + pk_columns: Vec, + blocked: Vec, +) -> DFResult { + if blocked.is_empty() || batch.num_rows() == 0 { + return Ok(batch); + } + let pk_indices = resolve_pk_indices(&batch, &pk_columns)?; + let to_df = |e: lance_core::Error| DataFusionError::Execution(e.to_string()); + + let mut keep = Vec::with_capacity(batch.num_rows()); + for row in 0..batch.num_rows() { + let values: Vec = pk_indices + .iter() + .map(|&col| ScalarValue::try_from_array(batch.column(col), row)) + .collect::>()?; + let on_disk_key = on_disk_pk_key(&values).map_err(to_df)?; + let mut blocked_row = false; + for membership in &blocked { + if membership + .contains(&values, &on_disk_key) + .await + .map_err(to_df)? + { + blocked_row = true; + break; + } + } + keep.push(!blocked_row); + } + filter_record_batch(&batch, &BooleanArray::from(keep)) + .map_err(|e| DataFusionError::ArrowError(Box::new(e), None)) +} + +impl Stream for PkBlockFilterStream { + type Item = DFResult; + + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let this = self.get_mut(); + loop { + // Drive an in-flight filter to completion before pulling more input. + if let Some(fut) = this.pending.as_mut() { + return match fut.as_mut().poll(cx) { + Poll::Ready(Ok(out)) => { + this.pending = None; + this.kept += out.num_rows(); + Poll::Ready(Some(Ok(out))) + } + Poll::Ready(Err(e)) => { + this.pending = None; + Poll::Ready(Some(Err(e))) + } + Poll::Pending => Poll::Pending, + }; + } + + match this.input.poll_next_unpin(cx) { + Poll::Ready(Some(Ok(batch))) => { + this.input_seen += batch.num_rows(); + this.pending = Some( + filter_batch(batch, this.pk_columns.clone(), this.blocked.clone()).boxed(), + ); + // Loop to poll the just-created future. + } + Poll::Ready(Some(Err(e))) => return Poll::Ready(Some(Err(e))), + Poll::Ready(None) => { + // >= k candidates in, < k out: over-fetch missed superseded rows. + if !this.warned && this.input_seen >= this.k && this.kept < this.k { + warn!( + k = this.k, + fetched = this.input_seen, + kept = this.kept, + "LSM vector search: < k live rows survived the PK post-filter; \ + raise the over-fetch factor or use a true KNN prefilter." + ); + this.warned = true; + } + return Poll::Ready(None); + } + Poll::Pending => return Poll::Pending, + } + } + } +} + +impl datafusion::physical_plan::RecordBatchStream for PkBlockFilterStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::dataset::mem_wal::write::{BatchStore, IndexStore}; + use arrow_array::Int32Array; + use arrow_schema::{DataType, Field, Schema}; + use datafusion::prelude::SessionContext; + use datafusion_physical_plan::test::TestMemoryExec; + use futures::TryStreamExt; + + fn int_batch(ids: &[i32]) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); + RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(ids.to_vec()))]).unwrap() + } + + /// An in-memory membership whose PK index holds `ids` (positions 0..n). + fn membership(ids: &[i32]) -> GenMembership { + let store = BatchStore::with_capacity(16); + let mut index = IndexStore::new(); + index.enable_pk_index(&[("id".to_string(), 0)]); + for &id in ids { + let b = int_batch(&[id]); + let (bp, off, _) = store.append(b.clone()).unwrap(); + index.insert_with_batch_position(&b, off, Some(bp)).unwrap(); + } + let max_visible_row = store.max_visible_row(index.max_visible_batch_position()); + GenMembership::InMemory { + index_store: Arc::new(index), + max_visible_row, + } + } + + async fn run(exec: PkBlockFilterExec) -> Vec { + let ctx = SessionContext::new(); + let out: Vec = exec + .execute(0, ctx.task_ctx()) + .unwrap() + .try_collect() + .await + .unwrap(); + out.iter() + .flat_map(|b| { + b.column_by_name("id") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap() + .values() + .to_vec() + }) + .collect() + } + + #[tokio::test] + async fn drops_rows_blocked_by_a_newer_generation() { + let b = int_batch(&[10, 20, 30]); + let input = TestMemoryExec::try_new_exec(&[vec![b.clone()]], b.schema(), None).unwrap(); + let exec = + PkBlockFilterExec::new(input, vec!["id".to_string()], vec![membership(&[20])], 1); + assert_eq!(run(exec).await, vec![10, 30]); + } + + #[tokio::test] + async fn blocks_a_pk_present_in_any_generation() { + // Two newer-gen memberships: a row is dropped if either contains its PK. + let b = int_batch(&[10, 20, 30]); + let blocked = vec![membership(&[10]), membership(&[30])]; + let input = TestMemoryExec::try_new_exec(&[vec![b.clone()]], b.schema(), None).unwrap(); + let exec = PkBlockFilterExec::new(input, vec!["id".to_string()], blocked, 1); + assert_eq!(run(exec).await, vec![20]); + } + + #[tokio::test] + async fn empty_blocked_keeps_all_rows() { + let b = int_batch(&[1, 2, 3]); + let input = TestMemoryExec::try_new_exec(&[vec![b.clone()]], b.schema(), None).unwrap(); + let exec = PkBlockFilterExec::new(input, vec!["id".to_string()], Vec::new(), 1); + assert_eq!(run(exec).await, vec![1, 2, 3]); + } +} diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs deleted file mode 100644 index d32b7993911..00000000000 --- a/rust/lance/src/dataset/mem_wal/scanner/exec/pk_hash_filter.rs +++ /dev/null @@ -1,366 +0,0 @@ -// SPDX-License-Identifier: Apache-2.0 -// SPDX-FileCopyrightText: Copyright The Lance Authors - -//! Drop superseded rows from a per-source KNN result by primary-key hash. -//! -//! Drops a row when its PK hash ([`super::compute_pk_hash`]) is in any `blocked` -//! set — the newer generations' membership (`Arc`, shared, never merged; -//! base table: all generations). Only the KNN output is hashed. -//! -//! Cross-generation only: within-gen duplicates share a hash, so the global -//! dedup's `(generation, freshness)` tiebreaker collapses those instead. -//! -//! Post-filters an over-fetched KNN (the planner's `overfetch_factor`); warns -//! when a source had >= k candidates but < k survived (over-fetch too small). - -use std::any::Any; -use std::fmt; -use std::pin::Pin; -use std::sync::Arc; -use std::task::{Context, Poll}; - -use arrow::compute::filter_record_batch; -use arrow_array::{BooleanArray, RecordBatch}; -use arrow_schema::SchemaRef; -use datafusion::common::ScalarValue; -use datafusion::error::{DataFusionError, Result as DFResult}; -use datafusion::execution::TaskContext; -use datafusion::physical_expr::EquivalenceProperties; -use datafusion::physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties, - SendableRecordBatchStream, -}; -use futures::{Stream, StreamExt}; -use tracing::warn; - -use super::super::block_list::GenMembership; -use super::pk::{compute_pk_hash, resolve_pk_indices}; - -/// Filters out rows whose PK is contained in any newer generation's membership. -#[derive(Debug)] -pub struct PkHashFilterExec { - input: Arc, - pk_columns: Vec, - /// Newer generations' membership; a row is blocked if any contains its hash. - /// In-memory generations are probed against their index; on-disk ones hold a - /// materialized hash set (see [`GenMembership`]). - blocked: Vec, - /// Target neighbor count, used only to warn on a per-source under-fetch. - k: usize, - properties: Arc, -} - -impl PkHashFilterExec { - pub fn new( - input: Arc, - pk_columns: Vec, - blocked: Vec, - k: usize, - ) -> Self { - // A filter preserves the input schema and partitioning. - let properties = Arc::new(PlanProperties::new( - EquivalenceProperties::new(input.schema()), - input.output_partitioning().clone(), - input.pipeline_behavior(), - input.boundedness(), - )); - Self { - input, - pk_columns, - blocked, - k, - properties, - } - } -} - -impl DisplayAs for PkHashFilterExec { - fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { - match t { - DisplayFormatType::Default - | DisplayFormatType::Verbose - | DisplayFormatType::TreeRender => { - write!( - f, - "PkHashFilterExec: pk_cols=[{}], gens={}", - self.pk_columns.join(", "), - self.blocked.len(), - ) - } - } - } -} - -impl ExecutionPlan for PkHashFilterExec { - fn name(&self) -> &str { - "PkHashFilterExec" - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - self.input.schema() - } - - fn properties(&self) -> &Arc { - &self.properties - } - - fn children(&self) -> Vec<&Arc> { - vec![&self.input] - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> DFResult> { - if children.len() != 1 { - return Err(DataFusionError::Internal( - "PkHashFilterExec requires exactly one child".to_string(), - )); - } - Ok(Arc::new(Self::new( - children[0].clone(), - self.pk_columns.clone(), - self.blocked.clone(), - self.k, - ))) - } - - fn execute( - &self, - partition: usize, - context: Arc, - ) -> DFResult { - let input_stream = self.input.execute(partition, context)?; - Ok(Box::pin(PkHashFilterStream { - input: input_stream, - pk_columns: self.pk_columns.clone(), - blocked: self.blocked.clone(), - k: self.k, - schema: self.schema(), - input_seen: 0, - kept: 0, - warned: false, - })) - } -} - -struct PkHashFilterStream { - input: SendableRecordBatchStream, - pk_columns: Vec, - blocked: Vec, - k: usize, - schema: SchemaRef, - input_seen: usize, - kept: usize, - warned: bool, -} - -impl PkHashFilterStream { - fn filter_batch(&self, batch: RecordBatch) -> DFResult { - if self.blocked.is_empty() || batch.num_rows() == 0 { - return Ok(batch); - } - let pk_indices = resolve_pk_indices(&batch, &self.pk_columns)?; - // On-disk generations probe by hash; in-memory ones probe their - // primary-key BTrees by value. Extract per-row values only when some - // membership needs them. - let needs_values = self.blocked.iter().any(GenMembership::needs_values); - let mut keep = Vec::with_capacity(batch.num_rows()); - for row in 0..batch.num_rows() { - let hash = compute_pk_hash(&batch, &pk_indices, row); - let values: Vec = if needs_values { - pk_indices - .iter() - .map(|&col| ScalarValue::try_from_array(batch.column(col), row)) - .collect::>()? - } else { - Vec::new() - }; - keep.push( - !self - .blocked - .iter() - .any(|membership| membership.contains(hash, &values)), - ); - } - filter_record_batch(&batch, &BooleanArray::from(keep)) - .map_err(|e| DataFusionError::ArrowError(Box::new(e), None)) - } -} - -impl Stream for PkHashFilterStream { - type Item = DFResult; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - match self.input.poll_next_unpin(cx) { - Poll::Ready(Some(Ok(batch))) => { - self.input_seen += batch.num_rows(); - match self.filter_batch(batch) { - Ok(out) => { - self.kept += out.num_rows(); - Poll::Ready(Some(Ok(out))) - } - Err(e) => Poll::Ready(Some(Err(e))), - } - } - Poll::Ready(None) => { - // >= k candidates in, < k out: the over-fetch missed superseded rows. - if !self.warned && self.input_seen >= self.k && self.kept < self.k { - warn!( - k = self.k, - fetched = self.input_seen, - kept = self.kept, - "LSM vector search: < k live rows survived the PK-hash post-filter; \ - raise the over-fetch factor or use a true KNN prefilter." - ); - self.warned = true; - } - Poll::Ready(None) - } - other => other, - } - } -} - -impl datafusion::physical_plan::RecordBatchStream for PkHashFilterStream { - fn schema(&self) -> SchemaRef { - self.schema.clone() - } -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow_array::{Int32Array, StringArray}; - use arrow_schema::{DataType, Field, Schema}; - use datafusion::prelude::SessionContext; - use datafusion_physical_plan::test::TestMemoryExec; - use futures::TryStreamExt; - use std::collections::HashSet; - - /// Hash a single-column Int32 PK value the way the exec does, so a test can - /// build blocked sets from values rather than hand-computed hashes. - fn hash_int_pk(id: i32) -> u64 { - let batch = int_batch(&[id]); - let pk_indices = resolve_pk_indices(&batch, &["id".to_string()]).unwrap(); - compute_pk_hash(&batch, &pk_indices, 0) - } - - fn int_batch(ids: &[i32]) -> RecordBatch { - let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)])); - RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(ids.to_vec()))]).unwrap() - } - - fn blocked(ids: &[i32]) -> Vec { - vec![GenMembership::Set(Arc::new( - ids.iter().map(|&id| hash_int_pk(id)).collect(), - ))] - } - - async fn run(exec: PkHashFilterExec) -> Vec { - let ctx = SessionContext::new(); - let out: Vec = exec - .execute(0, ctx.task_ctx()) - .unwrap() - .try_collect() - .await - .unwrap(); - out.iter() - .flat_map(|b| { - b.column_by_name("id") - .unwrap() - .as_any() - .downcast_ref::() - .unwrap() - .values() - .to_vec() - }) - .collect() - } - - #[tokio::test] - async fn drops_rows_with_blocked_pk_hash() { - let b = int_batch(&[10, 20, 30]); - let input = TestMemoryExec::try_new_exec(&[vec![b.clone()]], b.schema(), None).unwrap(); - let exec = PkHashFilterExec::new(input, vec!["id".to_string()], blocked(&[20]), 1); - assert_eq!(run(exec).await, vec![10, 30]); - } - - #[tokio::test] - async fn blocks_a_pk_present_in_any_generation_set() { - // Two newer-gen sets: a row is dropped if either contains its PK. - let b = int_batch(&[10, 20, 30]); - let sets = vec![ - GenMembership::Set(Arc::new(HashSet::from([hash_int_pk(10)]))), - GenMembership::Set(Arc::new(HashSet::from([hash_int_pk(30)]))), - ]; - let input = TestMemoryExec::try_new_exec(&[vec![b.clone()]], b.schema(), None).unwrap(); - let exec = PkHashFilterExec::new(input, vec!["id".to_string()], sets, 1); - assert_eq!(run(exec).await, vec![20]); - } - - #[tokio::test] - async fn empty_blocked_keeps_all_rows() { - let b = int_batch(&[1, 2, 3]); - let input = TestMemoryExec::try_new_exec(&[vec![b.clone()]], b.schema(), None).unwrap(); - let exec = PkHashFilterExec::new(input, vec!["id".to_string()], Vec::new(), 1); - assert_eq!(run(exec).await, vec![1, 2, 3]); - } - - #[tokio::test] - async fn null_pk_is_hashed_consistently_and_blockable() { - // A null PK hashes deterministically (compute_pk_hash hashes is_null), - // so a superseded null-key row can be dropped like any other. - let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, true)])); - let with_null = |ids: Vec>| { - RecordBatch::try_new(schema.clone(), vec![Arc::new(Int32Array::from(ids))]).unwrap() - }; - let pk = vec!["id".to_string()]; - let null_row = with_null(vec![None]); - let pk_indices = resolve_pk_indices(&null_row, &pk).unwrap(); - let sets = vec![GenMembership::Set(Arc::new(HashSet::from([ - compute_pk_hash(&null_row, &pk_indices, 0), - ])))]; - - // Rows: 10, NULL, 30 — only the NULL-key row is dropped. - let b = with_null(vec![Some(10), None, Some(30)]); - let input = TestMemoryExec::try_new_exec(&[vec![b.clone()]], b.schema(), None).unwrap(); - let exec = PkHashFilterExec::new(input, pk, sets, 1); - assert_eq!(run(exec).await, vec![10, 30]); - } - - #[tokio::test] - async fn composite_pk_hash_matches_block_set() { - // Composite PK (id, name): block the (2, "b") tuple only. - let schema = Arc::new(Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("name", DataType::Utf8, false), - ])); - let mk = |ids: &[i32], names: &[&str]| { - RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(Int32Array::from(ids.to_vec())), - Arc::new(StringArray::from(names.to_vec())), - ], - ) - .unwrap() - }; - let pk = vec!["id".to_string(), "name".to_string()]; - let one_row = mk(&[2], &["b"]); - let pk_indices = resolve_pk_indices(&one_row, &pk).unwrap(); - let sets = vec![GenMembership::Set(Arc::new(HashSet::from([ - compute_pk_hash(&one_row, &pk_indices, 0), - ])))]; - - // (1,"a") and (2,"a") survive; only the exact (2,"b") tuple is dropped. - let b = mk(&[1, 2, 2], &["a", "a", "b"]); - let input = TestMemoryExec::try_new_exec(&[vec![b.clone()]], b.schema(), None).unwrap(); - let exec = PkHashFilterExec::new(input, pk, sets, 1); - assert_eq!(run(exec).await, vec![1, 2]); - } -} diff --git a/rust/lance/src/dataset/mem_wal/scanner/flushed_cache.rs b/rust/lance/src/dataset/mem_wal/scanner/flushed_cache.rs index 39abf7e8c71..290c36dbb2d 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/flushed_cache.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/flushed_cache.rs @@ -22,11 +22,18 @@ use std::collections::HashSet; use std::sync::Arc; +use lance_core::cache::LanceCache; use lance_core::{Error, Result}; +use lance_index::scalar::ScalarIndex; use crate::dataset::{Dataset, DatasetBuilder}; use crate::session::Session; +/// Byte budget for the shared on-disk PK-index page cache. Generously sized: +/// PK BTree pages are small and re-probed across queries, so caching them keeps +/// the block-list probe cheap after the first touch. +const PK_INDEX_PAGE_CACHE_BYTES: usize = 256 * 1024 * 1024; + /// Cache of opened flushed-generation datasets, keyed by resolved path. /// /// Flushed generations live at a globally-unique, immutable path, so cached @@ -43,10 +50,13 @@ pub struct FlushedMemTableCache { // `try_get_with`, so concurrent first-queries on a just-flushed // generation open the dataset exactly once. inner: moka::future::Cache>, - // Per-generation set of PK hashes for the vector-search block-list, keyed by - // the same immutable flushed path. Built lazily on the first query that needs - // it (single-flight) so repeated searches skip re-scanning the PK column. - pk_hashes: moka::future::Cache>>, + // Opened standalone PK dedup index per flushed generation, keyed by the same + // immutable flushed path. Opened lazily on the first query that needs it + // (single-flight) so repeated searches reuse one `ScalarIndex`. + pk_indices: moka::future::Cache>, + // Shared page cache for the on-disk PK indexes' leaf pages, so probes after + // the first touch stay in memory instead of re-reading the object store. + index_page_cache: Arc, } impl FlushedMemTableCache { @@ -63,10 +73,11 @@ impl FlushedMemTableCache { // into at build time. .support_invalidation_closures() .build(), - pk_hashes: moka::future::Cache::builder() + pk_indices: moka::future::Cache::builder() .max_capacity(max_entries) .support_invalidation_closures() .build(), + index_page_cache: Arc::new(LanceCache::with_capacity(PK_INDEX_PAGE_CACHE_BYTES)), } } @@ -96,21 +107,27 @@ impl FlushedMemTableCache { .map_err(|e: Arc| Error::cloned(e.to_string())) } - /// Get the cached set of PK hashes for `path`, building it (exactly once) on - /// a miss via `build`. The flushed path is immutable, so a cached set is - /// never stale; concurrent first-queries share one build via `moka`'s + /// Get the opened PK dedup index for `path`, opening it (exactly once) on a + /// miss via `open`. The flushed path is immutable, so a cached index is + /// never stale; concurrent first-queries share one open via `moka`'s /// single-flight `try_get_with`. - pub async fn get_or_build_pk_hashes( + pub async fn get_or_open_pk_index( &self, path: &str, - build: impl std::future::Future>>, - ) -> Result>> { - self.pk_hashes - .try_get_with(path.to_string(), async move { build.await.map(Arc::new) }) + open: impl std::future::Future>>, + ) -> Result> { + self.pk_indices + .try_get_with(path.to_string(), open) .await .map_err(|e: Arc| Error::cloned(e.to_string())) } + /// The shared page cache to hand to the on-disk PK index loader so its leaf + /// pages are cached across probes. + pub fn index_page_cache(&self) -> Arc { + self.index_page_cache.clone() + } + /// Drop cached entries whose path is not in `live_paths`. /// /// Called by the consumer after compaction retires generations. Purely a @@ -127,7 +144,7 @@ impl FlushedMemTableCache { .invalidate_entries_if(move |path, _| !live.contains(path)); let live = live_paths.clone(); let _ = self - .pk_hashes + .pk_indices .invalidate_entries_if(move |path, _| !live.contains(path)); } } @@ -250,34 +267,6 @@ mod tests { assert_eq!(cache.inner.entry_count(), 1, "exactly one entry cached"); } - #[tokio::test] - async fn pk_hashes_cached_reuses_first_build() { - // The PK-hash set is keyed by the immutable flushed path: a hit returns - // the first-built set and never runs the second build closure. - let cache = FlushedMemTableCache::new(8); - let path = "memory://shard/gen_1"; - let first = cache - .get_or_build_pk_hashes(path, async { Ok(HashSet::from([1u64, 2])) }) - .await - .unwrap(); - let second = cache - .get_or_build_pk_hashes(path, async { - // Different contents; must be ignored because the path is cached. - Ok(HashSet::from([9u64])) - }) - .await - .unwrap(); - assert!( - Arc::ptr_eq(&first, &second), - "a PK-hash cache hit must reuse the first-built set" - ); - assert_eq!( - second.len(), - 2, - "cached set keeps the first build's contents" - ); - } - #[tokio::test] async fn test_retain_paths_drops_unreferenced() { let temp_dir = tempfile::tempdir().unwrap(); diff --git a/rust/lance/src/dataset/mem_wal/scanner/planner.rs b/rust/lance/src/dataset/mem_wal/scanner/planner.rs index f3f15e2e680..32cf9ed7880 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/planner.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/planner.rs @@ -15,7 +15,7 @@ use tracing::instrument; use super::collector::LsmDataSourceCollector; use super::data_source::LsmDataSource; -use super::exec::{MEMTABLE_GEN_COLUMN, MemtableGenTagExec, PkHashFilterExec, ROW_ADDRESS_COLUMN}; +use super::exec::{MEMTABLE_GEN_COLUMN, MemtableGenTagExec, PkBlockFilterExec, ROW_ADDRESS_COLUMN}; use super::flushed_cache::{FlushedMemTableCache, open_flushed_dataset}; use super::projection::{ build_scanner_projection, canonical_output_schema, null_columns, project_to_canonical, @@ -82,7 +82,7 @@ impl LsmScanPlanner { /// Each source is independently newest-per-PK (active via the fused /// [`MemTableDedupScanExec`](super::super::memtable::scanner), flushed via /// its within-generation deletion vector) and a cross-generation block-list - /// ([`PkHashFilterExec`]) drops any PK superseded by a newer generation. + /// ([`PkBlockFilterExec`]) drops any PK superseded by a newer generation. /// Each PK therefore survives in exactly one source, so a plain /// `UnionExec` carries at most one row per PK — no cross-source dedup, /// sort, or merge needed. `_memtable_gen` / `_rowaddr` are output-only and @@ -119,7 +119,6 @@ impl LsmScanPlanner { // `Box::pin` keeps the future off `clippy::large_futures`. let block_lists = Box::pin(super::block_list::compute_source_block_lists( &sources, - &self.pk_columns, self.session.as_ref(), self.flushed_cache.as_ref(), )) @@ -138,7 +137,7 @@ impl LsmScanPlanner { // Drop cross-generation stale rows (PKs superseded by a newer gen). // `k = 0`: there is no top-k, so the under-fetch warning never fires. let scan = match block_lists.get(&(source.shard_id(), source.generation())) { - Some(set) => Arc::new(PkHashFilterExec::new( + Some(set) => Arc::new(PkBlockFilterExec::new( scan, self.pk_columns.clone(), set.clone(), @@ -413,13 +412,36 @@ mod integration_tests { .unwrap() } - /// Create a dataset at the given URI with the provided batches. + /// Create a dataset at the given URI with the provided batches. Also writes + /// the standalone PK sidecar (on `id`) so a flushed-generation source can be + /// probed by the block-list; harmless for a base table (never probed). async fn create_dataset(uri: &str, batches: Vec) -> Dataset { let schema = batches[0].schema(); - let reader = RecordBatchIterator::new(batches.into_iter().map(Ok), schema); - Dataset::write(reader, uri, Some(WriteParams::default())) + let has_id = schema.column_with_name("id").is_some(); + let reader = RecordBatchIterator::new(batches.clone().into_iter().map(Ok), schema); + let dataset = Dataset::write(reader, uri, Some(WriteParams::default())) .await - .unwrap() + .unwrap(); + if has_id { + super::super::block_list::write_test_pk_sidecar(uri, &batches, &["id"]) + .await + .unwrap(); + } + dataset + } + + /// Build an in-memory memtable's `(batch_store, index_store)` with the PK + /// index enabled and populated (mirrors production — the block-list needs + /// the PK index to dedup in-memory generations). + fn pk_indexed(batches: &[RecordBatch]) -> (Arc, Arc) { + let batch_store = Arc::new(BatchStore::with_capacity(100)); + let mut index = IndexStore::new(); + index.enable_pk_index(&[("id".to_string(), 0)]); + for b in batches { + let (bp, off, _) = batch_store.append(b.clone()).unwrap(); + index.insert_with_batch_position(b, off, Some(bp)).unwrap(); + } + (batch_store, Arc::new(index)) } /// Setup a multi-level LSM structure with: @@ -470,10 +492,8 @@ mod integration_tests { .with_flushed_generation(2, "gen_2".to_string()); // Create active memtable - let batch_store = Arc::new(BatchStore::with_capacity(100)); - let index_store = Arc::new(IndexStore::new()); - let active_batch = create_test_batch(&schema, &[5, 6, 7], "active"); - let _ = batch_store.append(active_batch); + let (batch_store, index_store) = + pk_indexed(&[create_test_batch(&schema, &[5, 6, 7], "active")]); let active_memtable = InMemoryMemTables { active: InMemoryMemTableRef { @@ -515,18 +535,18 @@ mod integration_tests { // Verify the plan (gen DESC order: active -> gen2 -> gen1 -> base): // - plain UnionExec at top // - active arm: MemTableDedupScanExec (newest gen, not block-listed) - // - older arms: PkHashFilterExec (cross-gen block-list) -> LanceRead + // - older arms: PkBlockFilterExec (cross-gen block-list) -> LanceRead assert_plan_node_equals( plan, "ProjectionExec:... CoalescePartitionsExec UnionExec MemTableDedupScanExec: projection=[id, name, _rowaddr], with_row_id=false, with_row_address=true - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...gen_2... - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...gen_1... - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...base/data...refine_filter=--", ) .await @@ -549,9 +569,9 @@ mod integration_tests { // Verify the plan with `_memtable_gen` tags (gen DESC order): // - plain UnionExec at top - // - each arm: MemtableGenTagExec -> (PkHashFilterExec ->) data source + // - each arm: MemtableGenTagExec -> (PkBlockFilterExec ->) data source // - gen3 (active): MemtableGenTagExec -> MemTableDedupScanExec - // - gen2/gen1/base: MemtableGenTagExec -> PkHashFilterExec -> LanceRead + // - gen2/gen1/base: MemtableGenTagExec -> PkBlockFilterExec -> LanceRead assert_plan_node_equals( plan, "ProjectionExec:... @@ -560,13 +580,13 @@ mod integration_tests { MemtableGenTagExec: gen=gen3 MemTableDedupScanExec: projection=[id, name, _rowaddr], with_row_id=false, with_row_address=true MemtableGenTagExec: gen=gen2 - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...gen_2... MemtableGenTagExec: gen=gen1 - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...gen_1... MemtableGenTagExec: gen=base - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...base/data...refine_filter=--", ) .await @@ -647,14 +667,14 @@ mod integration_tests { } // base/gen1/gen2 all hold PKs superseded by a newer generation, so each - // is wrapped in a `PkHashFilterExec`; the newest (active) arm is not. + // is wrapped in a `PkBlockFilterExec`; the newest (active) arm is not. let plan = scanner.create_plan().await.unwrap(); let plan_str = format!( "{}", datafusion::physical_plan::displayable(plan.as_ref()).indent(true) ); assert!( - plan_str.contains("PkHashFilterExec"), + plan_str.contains("PkBlockFilterExec"), "filtered-read plan must apply the cross-gen block-list, got:\n{}", plan_str ); @@ -730,21 +750,21 @@ mod integration_tests { .with_flushed_generation(2, "gen_2".to_string()); // Frozen gen3 (sealed, NOT in the manifest) and active gen4. - let frozen_store = Arc::new(BatchStore::with_capacity(100)); - let _ = frozen_store.append(create_test_batch(&schema, &[6, 7], "frozen")); + let (frozen_store, frozen_index) = + pk_indexed(&[create_test_batch(&schema, &[6, 7], "frozen")]); let frozen = InMemoryMemTableRef { batch_store: frozen_store, - index_store: Arc::new(IndexStore::new()), + index_store: frozen_index, schema: schema.clone(), generation: 3, }; - let active_store = Arc::new(BatchStore::with_capacity(100)); - let _ = active_store.append(create_test_batch(&schema, &[7, 8], "active")); + let (active_store, active_index) = + pk_indexed(&[create_test_batch(&schema, &[7, 8], "active")]); let in_memory = InMemoryMemTables { active: InMemoryMemTableRef { batch_store: active_store, - index_store: Arc::new(IndexStore::new()), + index_store: active_index, schema: schema.clone(), generation: 4, }, @@ -969,12 +989,12 @@ mod integration_tests { ProjectionExec: expr=[id@0 as id, name@1 as name, NULL as _rowaddr] MemTableDedupScanExec: projection=[id, name, _rowaddr], with_row_id=false, with_row_address=true ProjectionExec: expr=[id@0 as id, name@1 as name, NULL as _rowaddr] - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...gen_2... ProjectionExec: expr=[id@0 as id, name@1 as name, NULL as _rowaddr] - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...gen_1... - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...base/data...refine_filter=--", ) .await @@ -1037,14 +1057,14 @@ mod integration_tests { MemTableDedupScanExec: projection=[id, name, _rowaddr], with_row_id=false, with_row_address=true MemtableGenTagExec: gen=gen2 ProjectionExec: expr=[id@0 as id, name@1 as name, NULL as _rowaddr] - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...gen_2... MemtableGenTagExec: gen=gen1 ProjectionExec: expr=[id@0 as id, name@1 as name, NULL as _rowaddr] - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...gen_1... MemtableGenTagExec: gen=base - PkHashFilterExec: pk_cols=[id]... + PkBlockFilterExec: pk_cols=[id]... LanceRead:...base/data...refine_filter=--", ) .await @@ -1113,6 +1133,8 @@ mod integration_tests { let mut index_store = IndexStore::new(); // Add BTree index on id column (field_id=0) index_store.add_btree("id_idx".to_string(), 0, "id".to_string()); + // Reuse it as the PK index so the block-list can dedup this generation. + index_store.enable_pk_index(&[("id".to_string(), 0)]); let active_batch = create_test_batch(&schema, &[5, 6, 7], "active"); let _ = batch_store.append(active_batch.clone()); @@ -1177,7 +1199,7 @@ mod integration_tests { // 1. Verify overall structure assert!(plan_str.contains("UnionExec"), "Should have UnionExec"); assert!( - plan_str.contains("PkHashFilterExec"), + plan_str.contains("PkBlockFilterExec"), "older generations should be block-list filtered" ); assert!( @@ -1365,7 +1387,6 @@ mod integration_tests { // Active memtable: id=10 inserted ("keep") then updated to NULL within // the same generation; id=20 ("active_20") is a control that matches. - let batch_store = Arc::new(BatchStore::with_capacity(16)); let active_batch = RecordBatch::try_new( schema.clone(), vec![ @@ -1378,12 +1399,12 @@ mod integration_tests { ], ) .unwrap(); - batch_store.append(active_batch).unwrap(); + let (batch_store, index_store) = pk_indexed(&[active_batch]); let in_memory = InMemoryMemTables { active: InMemoryMemTableRef { batch_store, - index_store: Arc::new(IndexStore::new()), + index_store, schema: schema.clone(), generation: 1, }, diff --git a/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs b/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs index 4036ef31d17..162d94b6c18 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/vector_search.rs @@ -41,7 +41,7 @@ use crate::session::Session; /// ([`super::exec::NewestPkFilterExec`], which drops a hit that isn't the newest /// visible version of its PK), flushed generations via their within-generation /// deletion vector — and the cross-generation block-list -/// ([`super::exec::PkHashFilterExec`]) drops any PK superseded by a newer +/// ([`super::exec::PkBlockFilterExec`]) drops any PK superseded by a newer /// generation. So each PK reaches the union from exactly one source and a /// distance-ordered merge yields the global top-k; no cross-source dedup is /// needed. @@ -59,11 +59,11 @@ use crate::session::Session; /// KNNExec: active memtable, fetch=ceil(k*overfetch) /// ProjectionExec (canonical output schema) /// ProjectionExec (null_columns _rowid) -/// PkHashFilterExec: block-list (flushed) +/// PkBlockFilterExec: block-list (flushed) /// KNNExec: flushed gen N, fetch=ceil(k*overfetch) (fast_search) /// … one per flushed gen … /// ProjectionExec (canonical output schema) -/// PkHashFilterExec: block-list (base) +/// PkBlockFilterExec: block-list (base) /// KNNExec: base table, k (fast_search)[.refine()?] /// ``` /// @@ -169,7 +169,7 @@ impl LsmVectorSearchPlanner { /// the rows that filtering drops: /// /// - `factor < 1.0` (e.g. `0.0`): **stale filtering off.** The per-source - /// block-list / [`super::exec::PkHashFilterExec`] is not built or applied, + /// block-list / [`super::exec::PkBlockFilterExec`] is not built or applied, /// so rows superseded by a newer generation can surface. The global PK /// dedup still runs, so it still suppresses stale copies in the cases /// where both the stale and the fresh row reach it. @@ -211,11 +211,10 @@ impl LsmVectorSearchPlanner { // live candidates after the post-filter. let overfetch_factor = overfetch_factor.max(1.0); - // Per-source PK-hash block sets (`NEWER(G)`; base = union of all gens). + // Per-source PK block sets (`NEWER(G)`; base = union of all gens). // `Box::pin` keeps the future off `clippy::large_futures`. let block_lists = Box::pin(super::block_list::compute_source_block_lists( &sources, - &self.pk_columns, self.session.as_ref(), self.flushed_cache.as_ref(), )) @@ -288,7 +287,7 @@ impl LsmVectorSearchPlanner { sort_by_distance(filtered, k)? } else { match blocked { - Some(set) => Arc::new(super::exec::PkHashFilterExec::new( + Some(set) => Arc::new(super::exec::PkBlockFilterExec::new( knn, self.pk_columns.clone(), set.clone(), @@ -377,7 +376,7 @@ impl LsmVectorSearchPlanner { merged_sorted }; - // Under-fetch is warned per-source inside `PkHashFilterExec`. + // Under-fetch is warned per-source inside `PkBlockFilterExec`. Ok(result) } @@ -586,10 +585,23 @@ mod tests { async fn create_dataset(uri: &str, batches: Vec) -> Dataset { let schema = batches[0].schema(); - let reader = RecordBatchIterator::new(batches.into_iter().map(Ok), schema); - Dataset::write(reader, uri, Some(WriteParams::default())) + let has_id = schema.column_with_name("id").is_some(); + let reader = RecordBatchIterator::new(batches.clone().into_iter().map(Ok), schema); + let dataset = Dataset::write(reader, uri, Some(WriteParams::default())) + .await + .unwrap(); + // Also write the standalone PK sidecar (on `id`) so a flushed-generation + // source can be probed by the block-list (harmless for a base table). + if has_id { + crate::dataset::mem_wal::scanner::block_list::write_test_pk_sidecar( + uri, + &batches, + &["id"], + ) .await - .unwrap() + .unwrap(); + } + dataset } #[tokio::test] @@ -1687,7 +1699,7 @@ mod tests { #[tokio::test] async fn test_vector_search_stale_read_when_fresh_falls_out_of_top_k() { // Regression for the cross-generation stale-read gap that the - // PkHashFilterExec block-list closes. + // PkBlockFilterExec block-list closes. // // Scenario: // * Base (gen 0): stale pk=1 sitting on the query (distance ~0). diff --git a/rust/lance/src/dataset/mem_wal/util.rs b/rust/lance/src/dataset/mem_wal/util.rs index d1413b84b2a..3f5090f6b40 100644 --- a/rust/lance/src/dataset/mem_wal/util.rs +++ b/rust/lance/src/dataset/mem_wal/util.rs @@ -169,6 +169,16 @@ pub fn flushed_memtable_path( shard_base_path(base_path, shard_id).join(format!("{}_gen_{}", random_hash, generation)) } +/// Subdirectory of a flushed generation holding its standalone primary-key +/// dedup index (a sidecar BTree, not registered in the manifest). Both the +/// flush writer and the block-list probe join this onto the generation path. +pub const PK_INDEX_DIR: &str = "_pk_index"; + +/// Path to a flushed generation's standalone primary-key dedup index. +pub fn pk_index_path(gen_path: &Path) -> Path { + gen_path.clone().join(PK_INDEX_DIR) +} + /// Generate an 8-character random hex string for flushed MemTable directories. pub fn generate_random_hash() -> String { let bytes: [u8; 4] = rand::random(); From afed6e199a86ec2fc73e7645f504dd87682a95a6 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Thu, 4 Jun 2026 06:43:21 -0500 Subject: [PATCH 10/12] refactor(mem_wal): route flushed PK index through the session index cache Aligns with jackye1995's review point #2 ("unified caching"): the flushed generation's standalone PK dedup index now loads through the opened dataset's session index cache (`dataset.index_cache.for_index(path)`, mirroring `open_scalar_index`) instead of a bespoke 256MB page cache + opened-index map inside `FlushedMemTableCache`. The opened index and its pages are cached and keyed by the immutable flushed path, with no separate cache path. Drops `FlushedMemTableCache::{pk_indices, index_page_cache, get_or_open_pk_index, index_page_cache}` and `PK_INDEX_PAGE_CACHE_BYTES`; the cache now holds only the opened datasets (which carry the session cache). mem_wal suite green; clippy --tests -D warnings clean. Co-Authored-By: Claude Opus 4.8 (1M context) --- .../src/dataset/mem_wal/scanner/block_list.rs | 69 +++++++------------ .../dataset/mem_wal/scanner/flushed_cache.rs | 48 +------------ 2 files changed, 29 insertions(+), 88 deletions(-) diff --git a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs index bd7253e3c7e..59cfd338723 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs @@ -18,7 +18,6 @@ use std::collections::HashMap; use std::sync::{Arc, LazyLock}; use datafusion::common::ScalarValue; -use lance_core::cache::LanceCache; use lance_core::{Error, Result}; use lance_index::metrics::NoOpMetricsCollector; @@ -240,62 +239,45 @@ fn in_memory_membership( } } -/// Open (or fetch the cached) standalone PK index for one flushed generation. -/// Cached by immutable path (single-flight); pages go through a shared cache. +/// Open the standalone PK BTree at `{flushed gen}/_pk_index` for one flushed +/// generation. Reuses the flushed dataset's (session-configured) object store +/// and **its index cache**, then loads the sidecar directly by path through the +/// BTree plugin — it is not a manifest index. The opened index and its pages +/// are cached in the session's index cache (keyed by the immutable flushed +/// path), so repeated probes reuse them with no separate cache path and no +/// upfront scan; concurrent first-opens may each load before the cache fills. async fn open_pk_index( path: &str, session: Option<&Arc>, flushed_cache: Option<&Arc>, -) -> Result> { - match flushed_cache { - Some(cache) => { - let page_cache = cache.index_page_cache(); - let build_path = path.to_string(); - let build_session = session.cloned(); - let build_cache = cache.clone(); - cache - .get_or_open_pk_index( - path, - // `Box::pin` keeps this open future off the caller's future - // (avoids `clippy::large_futures`). - Box::pin(async move { - build_pk_index( - &build_path, - build_session.as_ref(), - Some(&build_cache), - page_cache, - ) - .await - }), - ) - .await - } - None => build_pk_index(path, session, None, Arc::new(LanceCache::no_cache())).await, - } -} - -/// Open the standalone PK BTree at `{flushed gen}/_pk_index`. Reuses the flushed -/// dataset's (session-configured) object store, then loads the sidecar index -/// directly by path through the BTree plugin — it is not a manifest index. -async fn build_pk_index( - path: &str, - session: Option<&Arc>, - flushed_cache: Option<&Arc>, - page_cache: Arc, ) -> Result> { let dataset = open_flushed_dataset(path, session, flushed_cache).await?; + // Namespace the session index cache by the (immutable) flushed path so this + // sidecar's pages live alongside every other index instead of a bespoke + // cache. `fri_uuid` is None — flushed generations carry no fragment-reuse. + let index_cache = dataset.index_cache.for_index(path, None); let index_dir = dataset.base.clone().join(PK_INDEX_DIR); let store: Arc = Arc::new(LanceIndexStore::new( dataset.object_store.clone(), index_dir, - page_cache.clone(), + Arc::new(index_cache.clone()), )); + let plugin = PK_BTREE_REGISTRY.get_plugin_by_name("BTree")?; + // Cache the opened index in the session cache (mirrors `open_scalar_index`). + if let Some(index) = plugin + .get_from_cache(store.clone(), None, &index_cache) + .await? + { + return Ok(index); + } let details = prost_types::Any::from_msg(&lance_index::pbold::BTreeIndexDetails::default()) .map_err(|e| Error::io(e.to_string()))?; - plugin - .load_index(store, &details, None, page_cache.as_ref()) - .await + let index = plugin + .load_index(store, &details, None, &index_cache) + .await?; + plugin.put_in_cache(&index_cache, index.clone()).await?; + Ok(index) } /// Test helper: write a flushed generation's standalone PK sidecar at @@ -310,6 +292,7 @@ pub async fn write_test_pk_sidecar( pk_columns: &[&str], ) -> Result<()> { use datafusion::physical_plan::stream::RecordBatchStreamAdapter; + use lance_core::cache::LanceCache; use lance_index::scalar::btree::train_btree_index; use lance_io::object_store::ObjectStore; diff --git a/rust/lance/src/dataset/mem_wal/scanner/flushed_cache.rs b/rust/lance/src/dataset/mem_wal/scanner/flushed_cache.rs index 290c36dbb2d..0c2d3b039fe 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/flushed_cache.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/flushed_cache.rs @@ -22,18 +22,11 @@ use std::collections::HashSet; use std::sync::Arc; -use lance_core::cache::LanceCache; use lance_core::{Error, Result}; -use lance_index::scalar::ScalarIndex; use crate::dataset::{Dataset, DatasetBuilder}; use crate::session::Session; -/// Byte budget for the shared on-disk PK-index page cache. Generously sized: -/// PK BTree pages are small and re-probed across queries, so caching them keeps -/// the block-list probe cheap after the first touch. -const PK_INDEX_PAGE_CACHE_BYTES: usize = 256 * 1024 * 1024; - /// Cache of opened flushed-generation datasets, keyed by resolved path. /// /// Flushed generations live at a globally-unique, immutable path, so cached @@ -48,15 +41,10 @@ const PK_INDEX_PAGE_CACHE_BYTES: usize = 256 * 1024 * 1024; pub struct FlushedMemTableCache { // `moka`'s async cache gives a bounded size plus single-flight // `try_get_with`, so concurrent first-queries on a just-flushed - // generation open the dataset exactly once. + // generation open the dataset exactly once. The opened dataset carries the + // session index cache, which also backs each generation's standalone PK + // dedup index (see `block_list::open_pk_index`) — no separate cache path. inner: moka::future::Cache>, - // Opened standalone PK dedup index per flushed generation, keyed by the same - // immutable flushed path. Opened lazily on the first query that needs it - // (single-flight) so repeated searches reuse one `ScalarIndex`. - pk_indices: moka::future::Cache>, - // Shared page cache for the on-disk PK indexes' leaf pages, so probes after - // the first touch stay in memory instead of re-reading the object store. - index_page_cache: Arc, } impl FlushedMemTableCache { @@ -73,11 +61,6 @@ impl FlushedMemTableCache { // into at build time. .support_invalidation_closures() .build(), - pk_indices: moka::future::Cache::builder() - .max_capacity(max_entries) - .support_invalidation_closures() - .build(), - index_page_cache: Arc::new(LanceCache::with_capacity(PK_INDEX_PAGE_CACHE_BYTES)), } } @@ -107,27 +90,6 @@ impl FlushedMemTableCache { .map_err(|e: Arc| Error::cloned(e.to_string())) } - /// Get the opened PK dedup index for `path`, opening it (exactly once) on a - /// miss via `open`. The flushed path is immutable, so a cached index is - /// never stale; concurrent first-queries share one open via `moka`'s - /// single-flight `try_get_with`. - pub async fn get_or_open_pk_index( - &self, - path: &str, - open: impl std::future::Future>>, - ) -> Result> { - self.pk_indices - .try_get_with(path.to_string(), open) - .await - .map_err(|e: Arc| Error::cloned(e.to_string())) - } - - /// The shared page cache to hand to the on-disk PK index loader so its leaf - /// pages are cached across probes. - pub fn index_page_cache(&self) -> Arc { - self.index_page_cache.clone() - } - /// Drop cached entries whose path is not in `live_paths`. /// /// Called by the consumer after compaction retires generations. Purely a @@ -142,10 +104,6 @@ impl FlushedMemTableCache { let _ = self .inner .invalidate_entries_if(move |path, _| !live.contains(path)); - let live = live_paths.clone(); - let _ = self - .pk_indices - .invalidate_entries_if(move |path, _| !live.contains(path)); } } From 3147ae31740abc0430ddd0ae85e4dc1371fc0a81 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Thu, 4 Jun 2026 09:35:18 -0500 Subject: [PATCH 11/12] refactor(mem_wal): composite PK index is now a BTreeMemIndex, drop PkKeyIndex Per jackye1995's review #1 ("composite-key BTreeMemIndex"): instead of a separate parallel skiplist (`PkKeyIndex`), the composite PK index is a plain `BTreeMemIndex` keyed on a synthetic `Binary` column (`__pk_key__`) holding the order-preserving encoded tuple. `pk_key.rs` is now just the encoder (`encode_pk_tuple` + `encode_pk_batch`); the insert path materializes the encoded `Binary` column and feeds the existing index, and the probe seeks with `ScalarValue::Binary(encode_pk_tuple(values))`. Benefits: the composite case reuses `BTreeMemIndex`'s byte backend (incl. the inline-small-key node optimization) and its `to_training_batches`, so the in-memory probe, flush sidecar, and single-column path share one index type and one code path. Arity-split and its single-column memory/typed-fast-path wins are unchanged. Net deletion of the hand-rolled `PkKeyIndex` skiplist. mem_wal suite green; clippy --tests -D warnings clean. Co-Authored-By: Claude Opus 4.8 (1M context) --- rust/lance/src/dataset/mem_wal/index.rs | 54 +++-- .../lance/src/dataset/mem_wal/index/pk_key.rs | 212 +++--------------- 2 files changed, 71 insertions(+), 195 deletions(-) diff --git a/rust/lance/src/dataset/mem_wal/index.rs b/rust/lance/src/dataset/mem_wal/index.rs index c3cce7525e7..7c63c275e97 100644 --- a/rust/lance/src/dataset/mem_wal/index.rs +++ b/rust/lance/src/dataset/mem_wal/index.rs @@ -50,21 +50,27 @@ pub use fts::{FtsIndexConfig, FtsMemIndex, FtsQueryExpr, SearchOptions}; pub use hnsw::{HnswIndexConfig, HnswMemIndex}; pub use pk_key::encode_pk_tuple; -use pk_key::PkKeyIndex; +use pk_key::encode_pk_batch; + +/// Synthetic column the composite PK index is keyed on: the order-preserving +/// encoded tuple (see [`encode_pk_tuple`]), stored as `Binary` so a +/// [`BTreeMemIndex`]'s byte backend indexes it directly. +const PK_KEY_COLUMN: &str = "__pk_key__"; /// The memtable's primary-key index, used to answer "newest visible version of /// this key" for dedup. Single-column PKs reuse the column's compact typed -/// [`BTreeMemIndex`] (no second copy); composite PKs use one [`PkKeyIndex`] over -/// the order-preserving encoded tuple ([`encode_pk_tuple`]). Either way the -/// lookup is a single seek. +/// [`BTreeMemIndex`] (no second copy); composite PKs key a `BTreeMemIndex` on +/// the order-preserving encoded tuple ([`encode_pk_tuple`]) instead. Either way +/// the lookup is a single seek on one `BTreeMemIndex`. enum PkIndex { /// Arity 1: aliases a `btree_indexes` entry, so the insert loop maintains it. Single(Arc), - /// Arity >= 2: a dedicated encoded-tuple skiplist, maintained explicitly in - /// the insert paths. `columns` are the PK columns in order, resolved against - /// each batch's schema at insert time. + /// Arity >= 2: a `BTreeMemIndex` over the encoded-tuple `Binary` key, + /// maintained explicitly in the insert paths (the original batch lacks the + /// synthetic key column). `columns` are the PK columns in order, resolved + /// against each batch's schema at insert time. Composite { - index: Arc, + index: Arc, columns: Vec, }, } @@ -407,10 +413,11 @@ impl IndexStore { /// /// Single-column PKs reuse an existing BTree on the field, else auto-create /// one under a `__pk__*` name so the normal insert loop maintains it (no - /// second copy). Composite (arity >= 2) PKs build a dedicated encoded-tuple - /// [`PkKeyIndex`], maintained explicitly in the insert paths. Call once at - /// construction, after [`Self::from_configs`] and before any inserts; a - /// no-op when `pk_columns` is empty. + /// second copy). Composite (arity >= 2) PKs key a `BTreeMemIndex` on the + /// order-preserving encoded tuple (synthetic [`PK_KEY_COLUMN`]), maintained + /// explicitly in the insert paths. Call once at construction, after + /// [`Self::from_configs`] and before any inserts; a no-op when `pk_columns` + /// is empty. pub fn enable_pk_index(&mut self, pk_columns: &[(String, i32)]) { self.pk_index = match pk_columns { [] => None, @@ -431,7 +438,9 @@ impl IndexStore { Some(PkIndex::Single(btree)) } multi => Some(PkIndex::Composite { - index: Arc::new(PkKeyIndex::new()), + // Synthetic field id (-1): the composite index is held directly, + // never resolved by field id. + index: Arc::new(BTreeMemIndex::new(-1, PK_KEY_COLUMN.to_string())), columns: multi.iter().map(|(c, _)| c.clone()).collect(), }), }; @@ -471,11 +480,21 @@ impl IndexStore { .collect() } - /// Maintain the composite PK index for `batch` (no-op for single/no PK). + /// Maintain the composite PK index for `batch` (no-op for single/no PK): + /// encode the PK columns into the synthetic [`PK_KEY_COLUMN`] `Binary` column + /// and feed that to the keyed `BTreeMemIndex`. fn insert_composite_pk(&self, batch: &RecordBatch, row_offset: u64) -> Result<()> { if let Some(PkIndex::Composite { index, columns }) = &self.pk_index { let pk_indices = Self::pk_batch_indices(batch, columns)?; - index.insert(batch, &pk_indices, row_offset)?; + let encoded = encode_pk_batch(batch, &pk_indices)?; + let schema = Arc::new(arrow_schema::Schema::new(vec![arrow_schema::Field::new( + PK_KEY_COLUMN, + arrow_schema::DataType::Binary, + false, + )])); + let key_batch = RecordBatch::try_new(schema, vec![Arc::new(encoded)]) + .map_err(|e| Error::invalid_input(e.to_string()))?; + index.insert(&key_batch, row_offset)?; } Ok(()) } @@ -494,9 +513,10 @@ impl IndexStore { Some(PkIndex::Single(btree)) => btree.get_newest_visible(&values[0], max_visible_row), Some(PkIndex::Composite { index, .. }) => { // An unsupported PK type would have failed at insert, so the - // index can't hold a tuple this fails to encode. + // index can't hold a tuple this fails to encode. The probe key is + // the same `Binary`-encoded tuple the insert path indexed. let key = encode_pk_tuple(values).ok()?; - index.get_newest_visible(&key, max_visible_row) + index.get_newest_visible(&ScalarValue::Binary(Some(key)), max_visible_row) } } } diff --git a/rust/lance/src/dataset/mem_wal/index/pk_key.rs b/rust/lance/src/dataset/mem_wal/index/pk_key.rs index ce27f30b355..b31fe42c995 100644 --- a/rust/lance/src/dataset/mem_wal/index/pk_key.rs +++ b/rust/lance/src/dataset/mem_wal/index/pk_key.rs @@ -1,30 +1,22 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright The Lance Authors -//! Composite primary-key encoding + index for MemWAL dedup. +//! Composite primary-key encoding for MemWAL dedup. //! //! A multi-column primary key is reduced to a single order-preserving byte //! string ([`encode_pk_tuple`]) so the whole tuple is one comparable key: //! lexicographic byte order equals tuple order, and distinct tuples never -//! collide. The same key drives both the in-memory composite index -//! ([`PkKeyIndex`], a skiplist) and the flushed on-disk BTree (the key is the -//! index's `Binary` value column), so a probe builds `ScalarValue::Binary(key)` -//! and both ends agree. +//! collide. Encoded as a `Binary` value, the tuple is indexed directly by a +//! [`super::BTreeMemIndex`] (its byte backend) — both in memory and, after +//! flush, as the on-disk BTree's `Binary` value column — so a probe builds +//! `ScalarValue::Binary(key)` and every layer agrees. //! -//! Single-column primary keys do **not** use this — they keep the compact typed -//! BTree directly (see [`super::BTreeMemIndex`]). +//! Single-column primary keys do **not** use this — they key the typed +//! `BTreeMemIndex` on the column value directly. -use std::sync::Mutex; - -use arrow_array::{BinaryArray, RecordBatch, UInt64Array}; -use arrow_schema::{DataType, Field, Schema}; +use arrow_array::{BinaryArray, RecordBatch}; use datafusion::common::ScalarValue; -use lance_core::{Error, ROW_ID, Result}; -use lance_index::scalar::registry::VALUE_COLUMN_NAME; -use std::sync::Arc; - -use super::RowPosition; -use super::arena_skiplist::{SkipListReader, SkipListWriter, new_skiplist}; +use lance_core::{Error, Result}; /// Sign-flip a signed integer to an order-preserving unsigned key (matches the /// fixed-int BTree backend). Big-endian bytes of the result sort like the value. @@ -108,132 +100,24 @@ fn encode_pk_row(batch: &RecordBatch, pk_indices: &[usize], row: usize) -> Resul Ok(out) } -/// Skiplist key: the encoded tuple plus the row position (makes every entry -/// unique, so a non-unique key keeps every version). Sorts by `(bytes, pos)`. -#[derive(PartialEq, Eq)] -struct EncodedKey { - bytes: Box<[u8]>, - position: RowPosition, -} - -impl PartialOrd for EncodedKey { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - -impl Ord for EncodedKey { - fn cmp(&self, other: &Self) -> std::cmp::Ordering { - self.bytes - .cmp(&other.bytes) - .then(self.position.cmp(&other.position)) - } -} - -/// In-memory composite primary-key index: a skiplist over `(encoded_tuple, -/// position)`. Answers "newest visible version of this tuple" in one seek, the -/// composite analogue of [`super::BTreeMemIndex::get_newest_visible`]. -pub struct PkKeyIndex { - reader: SkipListReader, - writer: Mutex>, -} - -impl std::fmt::Debug for PkKeyIndex { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("PkKeyIndex") - .field("len", &self.len()) - .finish() +/// Encode every row of `batch`'s PK columns (at `pk_indices`) into a `Binary` +/// column of order-preserving composite keys — the form a [`super::BTreeMemIndex`] +/// indexes directly (its byte backend), so the composite PK reuses the same +/// index as a single-column one. +pub fn encode_pk_batch(batch: &RecordBatch, pk_indices: &[usize]) -> Result { + let mut keys: Vec> = Vec::with_capacity(batch.num_rows()); + for row in 0..batch.num_rows() { + keys.push(encode_pk_row(batch, pk_indices, row)?); } -} - -impl PkKeyIndex { - pub fn new() -> Self { - let (writer, reader) = new_skiplist::(); - Self { - reader, - writer: Mutex::new(writer), - } - } - - /// Insert every row of `batch`, encoding the PK columns at `pk_indices`. - /// `row_offset` is the absolute position of the first row. - pub fn insert(&self, batch: &RecordBatch, pk_indices: &[usize], row_offset: u64) -> Result<()> { - let mut writer = self.writer.lock().unwrap(); - for row in 0..batch.num_rows() { - let bytes = encode_pk_row(batch, pk_indices, row)?; - writer.insert(EncodedKey { - bytes: bytes.into_boxed_slice(), - position: row_offset + row as u64, - }); - } - Ok(()) - } - - /// Newest position of the pre-encoded tuple `key` visible at - /// `max_visible_row`, or `None`. A single seek-and-stop (no allocation). - pub fn get_newest_visible( - &self, - key: &[u8], - max_visible_row: RowPosition, - ) -> Option { - let target = EncodedKey { - bytes: key.into(), - position: max_visible_row, - }; - self.reader - .upper_bound_with(&target, |found| { - (found.bytes.as_ref() == key).then_some(found.position) - }) - .flatten() - } - - pub fn len(&self) -> usize { - self.reader.len() - } - - pub fn is_empty(&self) -> bool { - self.reader.len() == 0 - } - - /// Export as sorted `(Binary value, row_id)` batches to train the flushed - /// on-disk BTree. Entries are already in `(bytes, position)` order, so the - /// stream is sorted by value as `train_btree_index` requires. - pub fn to_training_batches(&self, batch_size: usize) -> Result> { - let schema = Arc::new(Schema::new(vec![ - Field::new(VALUE_COLUMN_NAME, DataType::Binary, true), - Field::new(ROW_ID, DataType::UInt64, false), - ])); - - let mut batches = Vec::new(); - let mut keys: Vec> = Vec::with_capacity(batch_size); - let mut row_ids: Vec = Vec::with_capacity(batch_size); - for entry in self.reader.iter() { - keys.push(entry.bytes.to_vec()); - row_ids.push(entry.position); - if keys.len() >= batch_size { - batches.push(build_batch(&schema, &keys, &row_ids)?); - keys.clear(); - row_ids.clear(); - } - } - if !keys.is_empty() { - batches.push(build_batch(&schema, &keys, &row_ids)?); - } - Ok(batches) - } -} - -fn build_batch(schema: &Arc, keys: &[Vec], row_ids: &[u64]) -> Result { - let values = BinaryArray::from_iter_values(keys.iter()); - let ids = UInt64Array::from(row_ids.to_vec()); - RecordBatch::try_new(schema.clone(), vec![Arc::new(values), Arc::new(ids)]) - .map_err(|e| Error::io(e.to_string())) + Ok(BinaryArray::from_iter_values(keys.iter())) } #[cfg(test)] mod tests { use super::*; use arrow_array::{Int32Array, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + use std::sync::Arc; fn tuple(a: i32, b: &str) -> Vec { vec![ScalarValue::Int32(Some(a)), ScalarValue::from(b)] @@ -294,55 +178,27 @@ mod tests { assert!(encode_pk_tuple(&empty).unwrap() < encode_pk_tuple(&with_zero).unwrap()); } - fn id_name_batch(ids: &[i32], names: &[&str]) -> RecordBatch { + #[test] + fn encode_pk_batch_matches_per_tuple_encoding() { + // Each row of the encoded `Binary` column equals `encode_pk_tuple` of + // that row's PK values — so the column a BTreeMemIndex indexes is exactly + // what a probe builds. let schema = Arc::new(Schema::new(vec![ Field::new("id", DataType::Int32, false), Field::new("name", DataType::Utf8, false), ])); - RecordBatch::try_new( + let batch = RecordBatch::try_new( schema, vec![ - Arc::new(Int32Array::from(ids.to_vec())), - Arc::new(StringArray::from(names.to_vec())), + Arc::new(Int32Array::from(vec![2, 1])), + Arc::new(StringArray::from(vec!["a", "b"])), ], ) - .unwrap() - } - - #[test] - fn pk_key_index_newest_visible_is_snapshot_bounded() { - let index = PkKeyIndex::new(); - // (1,"a")@0, (1,"b")@1, (1,"a")@2 — an update of (1,"a"). - index - .insert(&id_name_batch(&[1, 1, 1], &["a", "b", "a"]), &[0, 1], 0) - .unwrap(); - - let key_1a = encode_pk_tuple(&tuple(1, "a")).unwrap(); - let key_1b = encode_pk_tuple(&tuple(1, "b")).unwrap(); - // Newest visible (1,"a") is its re-write at position 2... - assert_eq!(index.get_newest_visible(&key_1a, 5), Some(2)); - // ...but bounded below the re-write, the older copy at 0. - assert_eq!(index.get_newest_visible(&key_1a, 1), Some(0)); - assert_eq!(index.get_newest_visible(&key_1b, 5), Some(1)); - // Absent tuple. - let key_2a = encode_pk_tuple(&tuple(2, "a")).unwrap(); - assert_eq!(index.get_newest_visible(&key_2a, 5), None); - } - - #[test] - fn training_batches_are_value_sorted() { - let index = PkKeyIndex::new(); - index - .insert(&id_name_batch(&[2, 1], &["a", "b"]), &[0, 1], 0) - .unwrap(); - let batches = index.to_training_batches(8192).unwrap(); - assert_eq!(batches.len(), 1); - let values = batches[0] - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - // (1,"b") encodes below (2,"a"), so it comes first. - assert!(values.value(0) < values.value(1)); + .unwrap(); + let encoded = encode_pk_batch(&batch, &[0, 1]).unwrap(); + assert_eq!(encoded.value(0), encode_pk_tuple(&tuple(2, "a")).unwrap()); + assert_eq!(encoded.value(1), encode_pk_tuple(&tuple(1, "b")).unwrap()); + // (1,"b") encodes below (2,"a"). + assert!(encoded.value(1) < encoded.value(0)); } } From edb899e2dff68ddcc64e087abdd50b3a51250e78 Mon Sep 17 00:00:00 2001 From: Daniel Rammer Date: Thu, 4 Jun 2026 09:54:36 -0500 Subject: [PATCH 12/12] refactor(mem_wal): collapse GenMembership::contains to a single key MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit After the composite-as-BTreeMemIndex change, the in-memory and flushed PK indexes share one key space, so the block-list probe no longer needs both the raw `values` and the pre-built `on_disk_key`. `GenMembership::contains` now takes one `key`, and `IndexStore::pk_contains_visible(values)` is replaced by `pk_contains_key(key)`, which forwards the already-built key straight to the keyed BTree (no re-encoding for composite). Callers build one key via `on_disk_pk_key`. (The two-variant `GenMembership` enum stays: in-memory is a sync, MVCC-bounded skiplist seek; flushed is an async Lance `ScalarIndex` search — genuinely different index types. The hash-set probe path jackye's review flagged is already gone.) mem_wal suite green; clippy --tests -D warnings clean. Co-Authored-By: Claude Opus 4.8 (1M context) --- rust/lance/src/dataset/mem_wal/index.rs | 30 ++++++++++++------- .../src/dataset/mem_wal/scanner/block_list.rs | 28 ++++++----------- .../src/dataset/mem_wal/scanner/builder.rs | 8 ++--- .../mem_wal/scanner/exec/pk_block_filter.rs | 8 ++--- 4 files changed, 34 insertions(+), 40 deletions(-) diff --git a/rust/lance/src/dataset/mem_wal/index.rs b/rust/lance/src/dataset/mem_wal/index.rs index 7c63c275e97..6f173109eff 100644 --- a/rust/lance/src/dataset/mem_wal/index.rs +++ b/rust/lance/src/dataset/mem_wal/index.rs @@ -534,15 +534,21 @@ impl IndexStore { self.pk_newest_visible(values, max_visible_row) == Some(position) } - /// Whether `values` has any version visible at `max_visible_row` — the + /// Whether `key` has any version visible at `max_visible_row` — the /// cross-source block-list's existence query, snapshot-bounded so a /// not-yet-visible write can't shadow an older visible copy. - pub fn pk_contains_visible( - &self, - values: &[ScalarValue], - max_visible_row: RowPosition, - ) -> bool { - self.pk_newest_visible(values, max_visible_row).is_some() + /// + /// `key` is already in the index's key space: the typed PK value for a + /// single-column key, the `Binary`-encoded tuple for a composite one (built + /// by `block_list::on_disk_pk_key`, the same key the flushed on-disk index is + /// probed with). Both arities forward it straight to the keyed BTree. + pub fn pk_contains_key(&self, key: &ScalarValue, max_visible_row: RowPosition) -> bool { + match &self.pk_index { + None => false, + Some(PkIndex::Single(btree)) | Some(PkIndex::Composite { index: btree, .. }) => { + btree.get_newest_visible(key, max_visible_row).is_some() + } + } } /// Whether the primary-key index holds no rows (or doesn't exist). @@ -932,8 +938,8 @@ mod tests { assert_eq!(store.pk_newest_visible(&one, 1), Some(0)); assert!(store.pk_is_newest(&one, 2, 5)); assert!(!store.pk_is_newest(&one, 0, 5)); - // Absent key. - assert!(!store.pk_contains_visible(&[ScalarValue::Int32(Some(9))], 5)); + // Absent key (probed by the typed value, as the block-list does). + assert!(!store.pk_contains_key(&ScalarValue::Int32(Some(9)), 5)); } #[test] @@ -965,9 +971,11 @@ mod tests { assert_eq!(store.pk_newest_visible(&tuple_1b, 5), Some(1)); // Watermark below the re-write: the older (1,"a")@0 is the newest visible. assert_eq!(store.pk_newest_visible(&tuple_1a, 1), Some(0)); - // An absent tuple. + // An absent tuple (probed by its Binary-encoded key, as the block-list + // does). let tuple_2a = [ScalarValue::Int32(Some(2)), ScalarValue::from("a")]; - assert!(!store.pk_contains_visible(&tuple_2a, 5)); + let key_2a = ScalarValue::Binary(Some(encode_pk_tuple(&tuple_2a).unwrap())); + assert!(!store.pk_contains_key(&key_2a, 5)); } #[test] diff --git a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs index 59cfd338723..1faccc6a6b5 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/block_list.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/block_list.rs @@ -56,28 +56,19 @@ pub enum GenMembership { } impl GenMembership { - /// Whether this generation visibly contains the primary key. The caller - /// supplies both the PK `values` (for the in-memory probe) and the - /// pre-built `on_disk_key` (typed value for a single-column PK, encoded - /// `Binary` tuple for a composite one — see [`on_disk_pk_key`]). - pub async fn contains( - &self, - values: &[ScalarValue], - on_disk_key: &ScalarValue, - ) -> Result { + /// Whether this generation visibly contains the primary `key` — the typed + /// value for a single-column PK, the encoded `Binary` tuple for a composite + /// one (built by [`on_disk_pk_key`]). The same key probes the in-memory + /// BTree and the flushed on-disk BTree, which now share a key space. + pub async fn contains(&self, key: &ScalarValue) -> Result { match self { Self::InMemory { index_store, max_visible_row, - } => { - Ok(max_visible_row.is_some_and(|max| index_store.pk_contains_visible(values, max))) - } + } => Ok(max_visible_row.is_some_and(|max| index_store.pk_contains_key(key, max))), Self::OnDisk(index) => { let result = index - .search( - &SargableQuery::Equals(on_disk_key.clone()), - &NoOpMetricsCollector, - ) + .search(&SargableQuery::Equals(key.clone()), &NoOpMetricsCollector) .await .map_err(|e| Error::io(e.to_string()))?; Ok(!search_is_empty(&result)) @@ -366,10 +357,9 @@ mod tests { /// Whether `id`'s PK is blocked by any of a source's newer-gen memberships. async fn blocks(memberships: &[GenMembership], id: i32) -> bool { - let values = [ScalarValue::Int32(Some(id))]; - let key = on_disk_pk_key(&values).unwrap(); + let key = on_disk_pk_key(&[ScalarValue::Int32(Some(id))]).unwrap(); for m in memberships { - if m.contains(&values, &key).await.unwrap() { + if m.contains(&key).await.unwrap() { return true; } } diff --git a/rust/lance/src/dataset/mem_wal/scanner/builder.rs b/rust/lance/src/dataset/mem_wal/scanner/builder.rs index 42db4d9694f..fb7087760c8 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/builder.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/builder.rs @@ -467,17 +467,17 @@ impl LsmScanner { .map_err(|e| Error::invalid_input(e.to_string()))?; let mut contained = Vec::with_capacity(pks.num_rows()); for row in 0..pks.num_rows() { - // In-memory generations probe by value; flushed ones probe their - // on-disk PK BTree with the typed/encoded key. + // Both in-memory and flushed generations probe by the same key (the + // typed value, or the encoded `Binary` tuple for a composite PK). let values: Vec = pk_indices .iter() .map(|&col| ScalarValue::try_from_array(pks.column(col), row)) .collect::>() .map_err(|e| Error::invalid_input(e.to_string()))?; - let on_disk_key = super::block_list::on_disk_pk_key(&values)?; + let key = super::block_list::on_disk_pk_key(&values)?; let mut found = false; for membership in &memberships { - if membership.contains(&values, &on_disk_key).await? { + if membership.contains(&key).await? { found = true; break; } diff --git a/rust/lance/src/dataset/mem_wal/scanner/exec/pk_block_filter.rs b/rust/lance/src/dataset/mem_wal/scanner/exec/pk_block_filter.rs index a3c4810d648..4eb703e99ab 100644 --- a/rust/lance/src/dataset/mem_wal/scanner/exec/pk_block_filter.rs +++ b/rust/lance/src/dataset/mem_wal/scanner/exec/pk_block_filter.rs @@ -191,14 +191,10 @@ async fn filter_batch( .iter() .map(|&col| ScalarValue::try_from_array(batch.column(col), row)) .collect::>()?; - let on_disk_key = on_disk_pk_key(&values).map_err(to_df)?; + let key = on_disk_pk_key(&values).map_err(to_df)?; let mut blocked_row = false; for membership in &blocked { - if membership - .contains(&values, &on_disk_key) - .await - .map_err(to_df)? - { + if membership.contains(&key).await.map_err(to_df)? { blocked_row = true; break; }