lance 4.0.0

A columnar data format that is 100x faster than Parquet for random access.
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

//! Point lookup planner for LSM scanner.
//!
//! Provides efficient primary key-based point lookups across LSM levels.

use std::sync::Arc;

use arrow_schema::SchemaRef;
use datafusion::common::ScalarValue;
use datafusion::physical_plan::ExecutionPlan;
use datafusion::physical_plan::limit::GlobalLimitExec;
use datafusion::prelude::Expr;
use lance_core::Result;
use lance_index::scalar::bloomfilter::sbbf::Sbbf;

use super::collector::LsmDataSourceCollector;
use super::data_source::LsmDataSource;
use super::exec::{BloomFilterGuardExec, CoalesceFirstExec, compute_pk_hash_from_scalars};

/// Plans point lookup queries over LSM data.
///
/// Point lookups are optimized for primary key-based queries where we expect
/// to find at most one row. The query plan uses:
///
/// 1. **Bloom filter guards**: Skip generations that definitely don't contain the key
/// 2. **Short-circuit evaluation**: Stop after finding the first match
/// 3. **Newest-first ordering**: Check newer generations before older ones
///
/// # Query Plan Structure
///
/// Since data is stored in reverse order (newest first), we use `GlobalLimitExec`
/// with limit=1 to take the first (most recent) matching row.
///
/// ```text
/// CoalesceFirstExec: return_first_non_null
///   BloomFilterGuardExec: gen=3
///     GlobalLimitExec: limit=1
///       FilterExec: pk = target
///         ScanExec: memtable_gen_3
///   BloomFilterGuardExec: gen=2
///     GlobalLimitExec: limit=1
///       FilterExec: pk = target
///         ScanExec: flushed_gen_2
///   BloomFilterGuardExec: gen=1
///     GlobalLimitExec: limit=1
///       FilterExec: pk = target
///         ScanExec: flushed_gen_1
///   GlobalLimitExec: limit=1
///     FilterExec: pk = target
///       ScanExec: base_table
/// ```
///
/// The base table doesn't use a bloom filter guard because:
/// - It's the fallback when no memtable has the key
/// - Bloom filters for the base table would be too large
pub struct LsmPointLookupPlanner {
    /// Data source collector.
    collector: LsmDataSourceCollector,
    /// Primary key column names.
    pk_columns: Vec<String>,
    /// Schema of the base table.
    base_schema: SchemaRef,
    /// Bloom filters for each memtable generation.
    /// Map: generation -> bloom filter
    bloom_filters: std::collections::HashMap<u64, Arc<Sbbf>>,
}

impl LsmPointLookupPlanner {
    /// Create a new planner.
    ///
    /// # Arguments
    ///
    /// * `collector` - Data source collector
    /// * `pk_columns` - Primary key column names
    /// * `base_schema` - Schema of the base table
    pub fn new(
        collector: LsmDataSourceCollector,
        pk_columns: Vec<String>,
        base_schema: SchemaRef,
    ) -> Self {
        Self {
            collector,
            pk_columns,
            base_schema,
            bloom_filters: std::collections::HashMap::new(),
        }
    }

    /// Add a bloom filter for a generation.
    ///
    /// Bloom filters are optional but improve performance by skipping
    /// generations that definitely don't contain the target key.
    pub fn with_bloom_filter(mut self, generation: u64, bloom_filter: Arc<Sbbf>) -> Self {
        self.bloom_filters.insert(generation, bloom_filter);
        self
    }

    /// Add multiple bloom filters.
    pub fn with_bloom_filters(
        mut self,
        bloom_filters: impl IntoIterator<Item = (u64, Arc<Sbbf>)>,
    ) -> Self {
        self.bloom_filters.extend(bloom_filters);
        self
    }

    /// Create a point lookup plan for the given primary key values.
    ///
    /// # Arguments
    ///
    /// * `pk_values` - Primary key values to look up (one value per pk column)
    /// * `projection` - Columns to include in output (None = all columns)
    ///
    /// # Returns
    ///
    /// An execution plan that returns at most one row - the newest version
    /// of the row with the given primary key.
    pub async fn plan_lookup(
        &self,
        pk_values: &[ScalarValue],
        projection: Option<&[String]>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        if pk_values.len() != self.pk_columns.len() {
            return Err(lance_core::Error::invalid_input(format!(
                "Expected {} primary key values, got {}",
                self.pk_columns.len(),
                pk_values.len()
            )));
        }

        let pk_hash = compute_pk_hash_from_scalars(pk_values);
        let filter_expr = self.build_pk_filter_expr(pk_values)?;
        let sources = self.collector.collect()?;

        if sources.is_empty() {
            return self.empty_plan(projection);
        }

        // Sort by generation DESC (newest first)
        let mut sources: Vec<_> = sources.into_iter().collect();
        sources.sort_by_key(|b| std::cmp::Reverse(b.generation()));

        let mut source_plans = Vec::new();

        for source in sources {
            let generation = source.generation().as_u64();

            let scan = self
                .build_source_scan(&source, projection, &filter_expr)
                .await?;

            // Data is stored in reverse order, so first match is newest
            let limited: Arc<dyn ExecutionPlan> = Arc::new(GlobalLimitExec::new(scan, 0, Some(1)));

            let guarded_plan: Arc<dyn ExecutionPlan> =
                if let Some(bf) = self.bloom_filters.get(&generation) {
                    Arc::new(BloomFilterGuardExec::new(
                        limited,
                        bf.clone(),
                        pk_hash,
                        generation,
                    ))
                } else {
                    limited
                };

            source_plans.push(guarded_plan);
        }

        let plan: Arc<dyn ExecutionPlan> = if source_plans.len() == 1 {
            source_plans.remove(0)
        } else {
            Arc::new(CoalesceFirstExec::new(source_plans))
        };

        Ok(plan)
    }

    /// Build the filter expression for primary key equality.
    fn build_pk_filter_expr(&self, pk_values: &[ScalarValue]) -> Result<Expr> {
        use datafusion::prelude::{col, lit};

        let mut expr: Option<Expr> = None;

        for (col_name, value) in self.pk_columns.iter().zip(pk_values.iter()) {
            let eq_expr = col(col_name.as_str()).eq(lit(value.clone()));

            expr = Some(match expr {
                Some(e) => e.and(eq_expr),
                None => eq_expr,
            });
        }

        expr.ok_or_else(|| lance_core::Error::invalid_input("No primary key columns specified"))
    }

    /// Build scan plan for a single data source.
    async fn build_source_scan(
        &self,
        source: &LsmDataSource,
        projection: Option<&[String]>,
        filter: &Expr,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        match source {
            LsmDataSource::BaseTable { dataset } => {
                let mut scanner = dataset.scan();
                let cols = self.build_projection(projection);
                scanner.project(&cols.iter().map(|s| s.as_str()).collect::<Vec<_>>())?;
                scanner.filter_expr(filter.clone());
                scanner.create_plan().await
            }
            LsmDataSource::FlushedMemTable { path, .. } => {
                let dataset = crate::dataset::DatasetBuilder::from_uri(path)
                    .load()
                    .await?;
                let mut scanner = dataset.scan();
                let cols = self.build_projection(projection);
                scanner.project(&cols.iter().map(|s| s.as_str()).collect::<Vec<_>>())?;
                scanner.filter_expr(filter.clone());
                scanner.create_plan().await
            }
            LsmDataSource::ActiveMemTable {
                batch_store,
                index_store,
                schema,
                ..
            } => {
                use crate::dataset::mem_wal::memtable::scanner::MemTableScanner;

                let mut scanner =
                    MemTableScanner::new(batch_store.clone(), index_store.clone(), schema.clone());
                if let Some(cols) = projection {
                    scanner.project(&cols.iter().map(|s| s.as_str()).collect::<Vec<_>>());
                }
                scanner.filter_expr(filter.clone());
                scanner.create_plan().await
            }
        }
    }

    /// Build projection list ensuring PK columns are included.
    fn build_projection(&self, projection: Option<&[String]>) -> Vec<String> {
        let mut cols: Vec<String> = if let Some(p) = projection {
            p.to_vec()
        } else {
            self.base_schema
                .fields()
                .iter()
                .map(|f| f.name().clone())
                .collect()
        };

        for pk in &self.pk_columns {
            if !cols.contains(pk) {
                cols.push(pk.clone());
            }
        }

        cols
    }

    /// Create an empty execution plan.
    fn empty_plan(&self, projection: Option<&[String]>) -> Result<Arc<dyn ExecutionPlan>> {
        use arrow_schema::{Field, Schema};
        use datafusion::physical_plan::empty::EmptyExec;

        let fields: Vec<Arc<Field>> = if let Some(cols) = projection {
            cols.iter()
                .filter_map(|name| {
                    self.base_schema
                        .field_with_name(name)
                        .ok()
                        .map(|f| Arc::new(f.clone()))
                })
                .collect()
        } else {
            self.base_schema.fields().iter().cloned().collect()
        };

        let schema = Arc::new(Schema::new(fields));
        Ok(Arc::new(EmptyExec::new(schema)))
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use arrow_array::{Int32Array, RecordBatch, RecordBatchIterator, StringArray};
    use arrow_schema::{DataType, Field, Schema as ArrowSchema};
    use datafusion::physical_plan::displayable;
    use std::collections::HashMap;
    use uuid::Uuid;

    use crate::dataset::mem_wal::scanner::data_source::RegionSnapshot;
    use crate::dataset::{Dataset, WriteParams};

    fn create_pk_schema() -> Arc<ArrowSchema> {
        let mut id_metadata = HashMap::new();
        id_metadata.insert(
            "lance-schema:unenforced-primary-key".to_string(),
            "true".to_string(),
        );
        let id_field = Field::new("id", DataType::Int32, false).with_metadata(id_metadata);

        Arc::new(ArrowSchema::new(vec![
            id_field,
            Field::new("name", DataType::Utf8, true),
        ]))
    }

    fn create_test_batch(schema: &ArrowSchema, ids: &[i32], name_prefix: &str) -> RecordBatch {
        let names: Vec<String> = ids
            .iter()
            .map(|id| format!("{}_{}", name_prefix, id))
            .collect();
        RecordBatch::try_new(
            Arc::new(schema.clone()),
            vec![
                Arc::new(Int32Array::from(ids.to_vec())),
                Arc::new(StringArray::from(names)),
            ],
        )
        .unwrap()
    }

    async fn create_dataset(uri: &str, batches: Vec<RecordBatch>) -> Dataset {
        let schema = batches[0].schema();
        let reader = RecordBatchIterator::new(batches.into_iter().map(Ok), schema);
        Dataset::write(reader, uri, Some(WriteParams::default()))
            .await
            .unwrap()
    }

    #[tokio::test]
    async fn test_point_lookup_plan_structure() {
        let schema = create_pk_schema();
        let temp_dir = tempfile::tempdir().unwrap();
        let base_path = temp_dir.path().to_str().unwrap();

        // Create base table
        let base_uri = format!("{}/base", base_path);
        let base_batch = create_test_batch(&schema, &[1, 2, 3], "base");
        let base_dataset = Arc::new(create_dataset(&base_uri, vec![base_batch]).await);

        // Create collector without memtables
        let collector = LsmDataSourceCollector::new(base_dataset, vec![]);

        let planner = LsmPointLookupPlanner::new(collector, vec!["id".to_string()], schema.clone());

        let pk_values = vec![ScalarValue::Int32(Some(2))];
        let plan = planner.plan_lookup(&pk_values, None).await.unwrap();

        // Verify plan structure
        let plan_str = format!("{}", displayable(plan.as_ref()).indent(true));

        // Should have GlobalLimitExec with limit=1 (data is stored in reverse order)
        assert!(
            plan_str.contains("GlobalLimitExec"),
            "Should have GlobalLimitExec in plan: {}",
            plan_str
        );
    }

    #[tokio::test]
    async fn test_point_lookup_with_memtables() {
        let schema = create_pk_schema();
        let temp_dir = tempfile::tempdir().unwrap();
        let base_path = temp_dir.path().to_str().unwrap();

        // Create base table
        let base_uri = format!("{}/base", base_path);
        let base_batch = create_test_batch(&schema, &[1, 2, 3], "base");
        let base_dataset = Arc::new(create_dataset(&base_uri, vec![base_batch]).await);

        // Create region snapshot
        let region_id = Uuid::new_v4();
        let gen1_uri = format!("{}/_mem_wal/{}/gen_1", base_uri, region_id);
        let gen1_batch = create_test_batch(&schema, &[2], "gen1"); // Update id=2
        create_dataset(&gen1_uri, vec![gen1_batch]).await;

        let region_snapshot = RegionSnapshot::new(region_id)
            .with_current_generation(2)
            .with_flushed_generation(1, "gen_1".to_string());

        // Create collector
        let collector = LsmDataSourceCollector::new(base_dataset, vec![region_snapshot]);

        let planner = LsmPointLookupPlanner::new(collector, vec!["id".to_string()], schema.clone());

        let pk_values = vec![ScalarValue::Int32(Some(2))];
        let plan = planner.plan_lookup(&pk_values, None).await.unwrap();

        // Verify plan structure - should have CoalesceFirstExec with multiple children
        let plan_str = format!("{}", displayable(plan.as_ref()).indent(true));

        assert!(
            plan_str.contains("CoalesceFirstExec") || plan_str.contains("GlobalLimitExec"),
            "Should have CoalesceFirstExec or GlobalLimitExec in plan: {}",
            plan_str
        );
    }

    #[tokio::test]
    async fn test_point_lookup_with_bloom_filter() {
        let schema = create_pk_schema();
        let temp_dir = tempfile::tempdir().unwrap();
        let base_path = temp_dir.path().to_str().unwrap();

        // Create base table
        let base_uri = format!("{}/base", base_path);
        let base_batch = create_test_batch(&schema, &[1, 2, 3], "base");
        let base_dataset = Arc::new(create_dataset(&base_uri, vec![base_batch]).await);

        // Create collector
        let collector = LsmDataSourceCollector::new(base_dataset, vec![]);

        // Create a bloom filter for generation 1 (simulating a memtable)
        let mut bf = Sbbf::with_ndv_fpp(100, 0.01).unwrap();
        let pk_hash = compute_pk_hash_from_scalars(&[ScalarValue::Int32(Some(2))]);
        bf.insert_hash(pk_hash);

        let planner = LsmPointLookupPlanner::new(collector, vec!["id".to_string()], schema.clone())
            .with_bloom_filter(1, Arc::new(bf));

        let pk_values = vec![ScalarValue::Int32(Some(2))];
        let plan = planner.plan_lookup(&pk_values, None).await.unwrap();

        // Plan should be valid
        assert!(plan.schema().field_with_name("id").is_ok());
    }

    #[tokio::test]
    async fn test_pk_filter_expr() {
        let schema = create_pk_schema();
        let temp_dir = tempfile::tempdir().unwrap();
        let base_uri = format!("{}/base", temp_dir.path().to_str().unwrap());
        let base_batch = create_test_batch(&schema, &[1], "base");
        let base_dataset = Arc::new(create_dataset(&base_uri, vec![base_batch]).await);

        let collector = LsmDataSourceCollector::new(base_dataset, vec![]);

        let planner = LsmPointLookupPlanner::new(collector, vec!["id".to_string()], schema);

        let pk_values = vec![ScalarValue::Int32(Some(42))];
        let expr = planner.build_pk_filter_expr(&pk_values).unwrap();

        // Verify expression is an equality
        let expr_str = format!("{}", expr);
        assert!(
            expr_str.contains("id"),
            "Expression should contain column name"
        );
    }
}