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
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors
//! In-memory MemTable for buffering writes.
pub mod batch_store;
pub mod flush;
pub mod scanner;
use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use std::time::{Duration, Instant};
use arrow_array::{Array, RecordBatch, RecordBatchIterator};
use arrow_schema::Schema as ArrowSchema;
use lance_core::datatypes::Schema;
use lance_core::{Error, Result};
use lance_index::scalar::bloomfilter::sbbf::Sbbf;
use tokio::sync::RwLock;
use uuid::Uuid;
use super::index::IndexStore;
use super::util::{WatchableOnceCell, WatchableOnceCellReader};
use super::write::{DurabilityResult, WalFlushResult};
use crate::Dataset;
use batch_store::BatchStore;
/// Default batch store capacity when not specified.
const DEFAULT_BATCH_CAPACITY: usize = 1024;
/// Configuration for the reader cache.
#[derive(Debug, Clone)]
pub struct CacheConfig {
/// Time-to-live for cached Dataset. Default: 60 seconds.
pub ttl: Duration,
/// Whether to always return fresh data (bypass cache). Default: false.
pub always_fresh: bool,
}
impl Default for CacheConfig {
fn default() -> Self {
Self {
ttl: Duration::from_secs(60),
always_fresh: false,
}
}
}
/// In-memory table for buffering writes.
///
/// Stores Arrow RecordBatches in a lock-free append-only structure for O(1) operations.
/// Dataset is constructed on-demand for reading with configurable caching.
///
/// # Thread Safety
///
/// - **Writer**: Only one thread should call `insert_with_seq()` at a time.
/// This is enforced by the WriteBatchHandler architecture.
/// - **Readers**: Multiple threads can safely call read methods concurrently.
pub struct MemTable {
/// Schema for this MemTable.
schema: Arc<ArrowSchema>,
/// Lance schema (for index operations).
lance_schema: Schema,
/// Lock-free batch storage.
/// Wrapped in Arc for sharing with scanners.
batch_store: Arc<BatchStore>,
/// Unique URI for on-demand Dataset construction.
dataset_uri: String,
/// Cache configuration for reading.
cache_config: CacheConfig,
/// Cached Dataset for reading (with eventual consistency).
cached_dataset: RwLock<Option<CachedDataset>>,
/// Generation number (incremented on flush).
generation: u64,
/// WAL batch mapping: batch_position -> (wal_entry_position, position within WAL entry).
wal_batch_mapping: HashMap<usize, (u64, usize)>,
/// Last WAL entry position that has been flushed.
last_flushed_wal_entry_position: u64,
/// Set of batch IDs that have been flushed to WAL.
flushed_batch_positions: HashSet<usize>,
/// Primary key bloom filter for staleness detection.
pk_bloom_filter: Sbbf,
/// Primary key field IDs (for bloom filter updates).
pk_field_ids: Vec<i32>,
/// Index registry (optional, for indexed writes).
/// Wrapped in Arc for sharing with async index handler.
indexes: Option<Arc<IndexStore>>,
/// WAL entry position when this memtable was frozen.
/// Used for WAL replay starting point during recovery.
/// None means the memtable is still active (not frozen).
frozen_at_wal_entry_position: Option<u64>,
/// Reader for WAL flush completion notification.
/// Set when the memtable is frozen and a WAL flush request is sent.
/// The reader can be awaited to know when WAL flush is complete.
/// Uses Mutex for interior mutability since the MemTable is wrapped in Arc when frozen.
/// Uses Result<WalFlushResult, String> since lance_core::Error doesn't implement Clone.
wal_flush_completion: std::sync::Mutex<
Option<WatchableOnceCellReader<std::result::Result<WalFlushResult, String>>>,
>,
/// Cell for memtable flush completion notification.
/// Created when the memtable is frozen and set with a value when the flush completes.
/// Used by backpressure to wait for oldest memtable flush completion.
memtable_flush_completion: std::sync::Mutex<Option<WatchableOnceCell<DurabilityResult>>>,
}
/// Cached Dataset with timestamp for eventual consistency.
struct CachedDataset {
dataset: Dataset,
created_at: Instant,
batch_count: usize,
}
/// Default expected items for primary key bloom filter.
/// Consistent with lance-index scalar bloomfilter defaults.
const PK_BLOOM_FILTER_EXPECTED_ITEMS: u64 = 8192;
/// Default false positive probability for primary key bloom filter.
/// Consistent with lance-index scalar bloomfilter defaults (≈ 1 in 1754).
const PK_BLOOM_FILTER_FPP: f64 = 0.00057;
impl MemTable {
/// Create a new MemTable with default capacity.
///
/// # Arguments
///
/// * `schema` - Arrow schema for the data
/// * `generation` - Initial generation number (typically 1 for new, or from recovery)
/// * `pk_field_ids` - Field IDs that form the primary key (for bloom filter)
pub fn new(schema: Arc<ArrowSchema>, generation: u64, pk_field_ids: Vec<i32>) -> Result<Self> {
Self::with_capacity(
schema,
generation,
pk_field_ids,
CacheConfig::default(),
DEFAULT_BATCH_CAPACITY,
)
}
/// Create a new MemTable with custom cache configuration.
///
/// # Arguments
///
/// * `schema` - Arrow schema for the data
/// * `generation` - Initial generation number (typically 1 for new, or from recovery)
/// * `pk_field_ids` - Field IDs that form the primary key (for bloom filter)
/// * `cache_config` - Configuration for reader cache (TTL, freshness)
pub fn with_cache_config(
schema: Arc<ArrowSchema>,
generation: u64,
pk_field_ids: Vec<i32>,
cache_config: CacheConfig,
) -> Result<Self> {
Self::with_capacity(
schema,
generation,
pk_field_ids,
cache_config,
DEFAULT_BATCH_CAPACITY,
)
}
/// Create a new MemTable with custom capacity.
///
/// # Arguments
///
/// * `schema` - Arrow schema for the data
/// * `generation` - Initial generation number (typically 1 for new, or from recovery)
/// * `pk_field_ids` - Field IDs that form the primary key (for bloom filter)
/// * `cache_config` - Configuration for reader cache (TTL, freshness)
/// * `batch_capacity` - Maximum number of batches before flush is required
pub fn with_capacity(
schema: Arc<ArrowSchema>,
generation: u64,
pk_field_ids: Vec<i32>,
cache_config: CacheConfig,
batch_capacity: usize,
) -> Result<Self> {
let lance_schema = Schema::try_from(schema.as_ref())?;
// Initialize bloom filter for primary key staleness detection.
let pk_bloom_filter =
Sbbf::with_ndv_fpp(PK_BLOOM_FILTER_EXPECTED_ITEMS, PK_BLOOM_FILTER_FPP).map_err(
|e| {
Error::io(format!(
"Failed to create bloom filter for primary key: {}",
e
))
},
)?;
// Generate unique URI for on-demand Dataset construction
let dataset_uri = format!("memory://{}", Uuid::new_v4());
// Create lock-free batch store
let batch_store = Arc::new(BatchStore::with_capacity(batch_capacity));
// Create memtable_flush_completion cell immediately so backpressure can
// wait on it even before the memtable is frozen. Every memtable will
// eventually be frozen and flushed.
let memtable_flush_cell = WatchableOnceCell::new();
Ok(Self {
schema,
lance_schema,
batch_store,
dataset_uri,
cache_config,
cached_dataset: RwLock::new(None),
generation,
wal_batch_mapping: HashMap::new(),
last_flushed_wal_entry_position: 0,
flushed_batch_positions: HashSet::new(),
pk_bloom_filter,
pk_field_ids,
indexes: None,
frozen_at_wal_entry_position: None,
wal_flush_completion: std::sync::Mutex::new(None),
memtable_flush_completion: std::sync::Mutex::new(Some(memtable_flush_cell)),
})
}
/// Set the index registry for indexed writes.
pub fn set_indexes(&mut self, indexes: IndexStore) {
self.indexes = Some(Arc::new(indexes));
}
/// Set the index registry with an Arc (for sharing with async handler).
pub fn set_indexes_arc(&mut self, indexes: Arc<IndexStore>) {
self.indexes = Some(indexes);
}
/// Mark this memtable as frozen with the given WAL entry position.
///
/// Once frozen, no new writes should be added. The memtable will be
/// added to the immutable queue for flushing to Lance storage.
///
/// # Arguments
///
/// * `wal_entry_position` - The last WAL entry position when this memtable was frozen
pub fn freeze(&mut self, wal_entry_position: u64) {
self.frozen_at_wal_entry_position = Some(wal_entry_position);
}
/// Set the WAL flush completion reader.
///
/// Called when a WAL flush request is sent at freeze time.
/// The reader can be awaited by flush_oldest_immutable to know when
/// the WAL flush is complete.
pub fn set_wal_flush_completion(
&self,
reader: WatchableOnceCellReader<std::result::Result<WalFlushResult, String>>,
) {
*self.wal_flush_completion.lock().unwrap() = Some(reader);
}
/// Take the WAL flush completion reader.
///
/// Returns the reader if set, consuming it. Used by flush_oldest_immutable
/// to await WAL flush completion before proceeding with memtable flush.
/// Thread-safe via interior mutability.
pub fn take_wal_flush_completion(
&self,
) -> Option<WatchableOnceCellReader<std::result::Result<WalFlushResult, String>>> {
self.wal_flush_completion.lock().unwrap().take()
}
/// Check if this memtable has a pending WAL flush completion to await.
pub fn has_pending_wal_flush(&self) -> bool {
self.wal_flush_completion.lock().unwrap().is_some()
}
/// Get a reader for the memtable flush completion.
///
/// The cell is created at memtable construction time, so this always
/// returns a reader. This allows backpressure to wait on the active
/// memtable's flush completion, not just frozen memtables.
///
/// # Panics
///
/// Panics if called after `signal_memtable_flush_complete()` has consumed the cell.
pub fn create_memtable_flush_completion(&self) -> WatchableOnceCellReader<DurabilityResult> {
self.memtable_flush_completion
.lock()
.unwrap()
.as_ref()
.expect("memtable_flush_completion cell should exist (created at construction)")
.reader()
}
/// Get a reader for the memtable flush completion.
///
/// Returns a reader if the completion cell exists, without consuming it.
/// Multiple readers can be obtained from the same cell.
pub fn get_memtable_flush_watcher(&self) -> Option<WatchableOnceCellReader<DurabilityResult>> {
self.memtable_flush_completion
.lock()
.unwrap()
.as_ref()
.map(|cell| cell.reader())
}
/// Signal that the memtable flush is complete.
///
/// Called after the memtable has been flushed to Lance storage.
pub fn signal_memtable_flush_complete(&self) {
if let Some(cell) = self.memtable_flush_completion.lock().unwrap().take() {
cell.write(DurabilityResult::ok());
}
}
/// Get the WAL entry position when this memtable was frozen.
///
/// Returns `None` if the memtable is still active (not frozen).
pub fn frozen_at_wal_entry_position(&self) -> Option<u64> {
self.frozen_at_wal_entry_position
}
/// Check if this memtable has been frozen.
pub fn is_frozen(&self) -> bool {
self.frozen_at_wal_entry_position.is_some()
}
/// Insert a record batch into the MemTable.
///
/// O(1) append.
///
/// # Returns
///
/// The batch position (0-indexed) for the inserted batch.
///
/// # Single Writer Requirement
///
/// This method MUST only be called from the single writer task.
pub async fn insert(&mut self, batch: RecordBatch) -> Result<usize> {
// Validate schema compatibility
if batch.schema() != self.schema {
return Err(Error::invalid_input(
"Batch schema doesn't match MemTable schema",
));
}
let num_rows = batch.num_rows();
if num_rows == 0 {
return Err(Error::invalid_input("Cannot insert empty batch"));
}
// Row offset is the current row count (before adding this batch)
let row_offset = self.batch_store.total_rows() as u64;
// Update bloom filter with primary keys
self.update_bloom_filter(&batch)?;
// Get batch position before appending (for index coverage tracking)
let batch_position = self.batch_store.len();
// Update indexes with batch position for coverage tracking
if let Some(ref indexes) = self.indexes {
indexes.insert_with_batch_position(&batch, row_offset, Some(batch_position))?;
}
// Append to batch store (returns batch_position, row_offset, estimated_size)
let (batch_position, _row_offset, _estimated_size) =
self.batch_store.append(batch).map_err(|_| {
Error::invalid_input("MemTable batch store is full - should have been flushed")
})?;
Ok(batch_position)
}
/// Insert a batch without updating indexes.
///
/// Index updates are performed during WAL flush by `WalFlushHandler`.
///
/// Returns `(batch_position, row_offset, estimated_size)` so the caller can queue the index update.
///
/// # Single Writer Requirement
///
/// This method MUST only be called from the single writer task.
pub async fn insert_batch_only(&mut self, batch: RecordBatch) -> Result<(usize, u64, usize)> {
// Validate schema compatibility
if batch.schema() != self.schema {
return Err(Error::invalid_input(
"Batch schema doesn't match MemTable schema",
));
}
let num_rows = batch.num_rows();
if num_rows == 0 {
return Err(Error::invalid_input("Cannot insert empty batch"));
}
// Update bloom filter with primary keys
self.update_bloom_filter(&batch)?;
// NOTE: Index update is skipped - caller will queue async update
// Append to batch store (returns batch_position, row_offset, estimated_size)
let (batch_position, row_offset, estimated_size) =
self.batch_store.append(batch).map_err(|_| {
Error::invalid_input("MemTable batch store is full - should have been flushed")
})?;
Ok((batch_position, row_offset, estimated_size))
}
/// Insert multiple batches without updating indexes.
///
/// All batches are inserted atomically - readers see either none or all.
/// Index updates are performed during WAL flush by `WalFlushHandler`.
///
/// Returns `Vec<(batch_position, row_offset, estimated_size)>` for each batch.
///
/// # Single Writer Requirement
///
/// This method MUST only be called from the single writer task.
pub async fn insert_batches_only(
&mut self,
batches: Vec<RecordBatch>,
) -> Result<Vec<(usize, u64, usize)>> {
if batches.is_empty() {
return Ok(vec![]);
}
// Validate all batches upfront
for (i, batch) in batches.iter().enumerate() {
if batch.schema() != self.schema {
return Err(Error::invalid_input(format!(
"Batch {} schema doesn't match MemTable schema",
i
)));
}
if batch.num_rows() == 0 {
return Err(Error::invalid_input(format!("Batch {} is empty", i)));
}
}
// Update bloom filter for all batches
for batch in &batches {
self.update_bloom_filter(batch)?;
}
// NOTE: Index update is skipped - caller will queue async update
// Append all batches atomically
let results = self.batch_store.append_batches(batches).map_err(|_| {
Error::invalid_input("MemTable batch store is full - should have been flushed")
})?;
Ok(results)
}
/// Check if the MemTable should be flushed.
///
/// Returns true if the batch store is full or estimated size exceeds threshold.
pub fn should_flush(&self, max_bytes: usize) -> bool {
self.batch_store.is_full() || self.batch_store.estimated_bytes() >= max_bytes
}
/// Get batches visible up to a specific batch position (inclusive).
///
/// A batch at position `i` is visible if `i <= max_visible_batch_position`.
///
/// # Arguments
///
/// * `max_visible_batch_position` - The maximum batch position to include (inclusive)
///
/// # Returns
///
/// Vector of visible batches.
pub async fn get_visible_batches(&self, max_visible_batch_position: usize) -> Vec<RecordBatch> {
self.batch_store
.visible_record_batches(max_visible_batch_position)
}
/// Get batch positions visible up to a specific batch position (inclusive).
///
/// This is useful for filtering index results by visibility.
pub async fn get_max_visible_batch_positions(
&self,
max_visible_batch_position: usize,
) -> Vec<usize> {
self.batch_store
.max_visible_batch_positions(max_visible_batch_position)
}
/// Check if a specific batch is visible at a given visibility position.
///
/// Returns true if the batch is visible, false if not visible or doesn't exist.
pub async fn is_batch_visible(
&self,
batch_position: usize,
max_visible_batch_position: usize,
) -> bool {
self.batch_store
.is_batch_visible(batch_position, max_visible_batch_position)
}
/// Scan batches visible up to a specific batch position.
///
/// This combines `get_visible_batches` with the scan interface.
pub async fn scan_batches_at_position(
&self,
max_visible_batch_position: usize,
) -> Result<Vec<RecordBatch>> {
Ok(self.get_visible_batches(max_visible_batch_position).await)
}
/// Update the bloom filter with primary keys from a batch.
fn update_bloom_filter(&mut self, batch: &RecordBatch) -> Result<()> {
let bloom = &mut self.pk_bloom_filter;
// Get primary key columns
let pk_columns: Vec<_> = self
.pk_field_ids
.iter()
.filter_map(|&field_id| {
// Find column by field ID
self.lance_schema
.fields
.iter()
.position(|f| f.id == field_id)
.and_then(|idx| batch.column(idx).clone().into())
})
.collect();
if pk_columns.len() != self.pk_field_ids.len() {
return Err(Error::invalid_input("Batch is missing primary key columns"));
}
// Insert each row's primary key hash
for row_idx in 0..batch.num_rows() {
let hash = compute_row_hash(&pk_columns, row_idx);
bloom.insert_hash(hash);
}
Ok(())
}
/// Mark batches as flushed to WAL.
///
/// Updates the WAL batch mapping for use during MemTable flush.
/// Also updates the batch_store's watermark to the highest flushed batch_position.
pub fn mark_wal_flushed(
&mut self,
batch_positions: &[usize],
wal_entry_position: u64,
positions: &[usize],
) {
for (idx, &batch_position) in batch_positions.iter().enumerate() {
self.wal_batch_mapping
.insert(batch_position, (wal_entry_position, positions[idx]));
self.flushed_batch_positions.insert(batch_position);
}
self.last_flushed_wal_entry_position = wal_entry_position;
// Update batch_store watermark to the highest batch_position flushed (inclusive)
if let Some(&max_batch_position) = batch_positions.iter().max() {
self.batch_store
.set_max_flushed_batch_position(max_batch_position);
}
}
/// Get or create a Dataset for reading.
///
/// Uses caching based on the configured eventual consistency strategy:
/// - If `always_fresh` is true, always constructs a new Dataset
/// - Otherwise, returns cached Dataset if within TTL and has same batch count
///
/// Returns None if there's no data to read.
pub async fn get_or_create_dataset(&self) -> Result<Option<Dataset>> {
let current_batch_count = self.batch_count();
if current_batch_count == 0 {
return Ok(None);
}
// Check if we can use cached dataset
if !self.cache_config.always_fresh {
let cached = self.cached_dataset.read().await;
if let Some(ref cached_ds) = *cached {
// Check if cache is still valid (within TTL and same batch count)
if cached_ds.batch_count == current_batch_count
&& cached_ds.created_at.elapsed() < self.cache_config.ttl
{
return Ok(Some(cached_ds.dataset.clone()));
}
}
}
// Need to construct a new Dataset
let dataset = self.construct_dataset().await?;
// Cache the new dataset (unless always_fresh)
if !self.cache_config.always_fresh {
let mut cached = self.cached_dataset.write().await;
*cached = Some(CachedDataset {
dataset: dataset.clone(),
created_at: Instant::now(),
batch_count: current_batch_count,
});
}
Ok(Some(dataset))
}
/// Construct a fresh Dataset from stored batches.
async fn construct_dataset(&self) -> Result<Dataset> {
if self.batch_store.is_empty() {
return Err(Error::invalid_input("Cannot construct Dataset: no batches"));
}
// Get batches
let batches = self.batch_store.to_vec();
// Create a new Dataset with all the batches
let reader = RecordBatchIterator::new(batches.into_iter().map(Ok), self.schema.clone());
let dataset = Dataset::write(reader, &self.dataset_uri, None).await?;
Ok(dataset)
}
/// Scan all data from the MemTable.
///
/// Returns all batches for flushing to persistent storage.
pub async fn scan_batches(&self) -> Result<Vec<RecordBatch>> {
Ok(self.batch_store.to_vec())
}
/// Scan all data from the MemTable in reverse order (newest first).
///
/// This is used when flushing MemTable to persistent storage to ensure
/// the flushed data is ordered from newest to oldest. This enables more
/// efficient K-way merge during LSM scan because flushed generations
/// will be pre-sorted in the order needed for deduplication.
///
/// The total number of rows in the MemTable is also returned to allow
/// callers to compute reversed row positions for indexes.
pub async fn scan_batches_reversed(&self) -> Result<(Vec<RecordBatch>, usize)> {
let total_rows = self.batch_store.total_rows();
let batches = self.batch_store.to_vec_reversed()?;
Ok((batches, total_rows))
}
/// Scan specific batches by their batch_positions.
pub async fn scan_batches_by_ids(&self, batch_positions: &[usize]) -> Result<Vec<RecordBatch>> {
let mut results = Vec::with_capacity(batch_positions.len());
for &batch_position in batch_positions {
let batch = self.batch_store.get_batch(batch_position).ok_or_else(|| {
Error::invalid_input(format!("Batch {} not found", batch_position))
})?;
results.push(batch.clone());
}
Ok(results)
}
/// Get batches for WAL flush.
pub async fn get_batches_for_wal(&self, batch_positions: &[usize]) -> Result<Vec<RecordBatch>> {
self.scan_batches_by_ids(batch_positions).await
}
/// Check if a primary key might exist in this MemTable.
///
/// Uses bloom filter for fast negative lookups.
/// Returns true if the key might exist, false if definitely not present.
pub fn might_contain_pk(&self, pk_hash: u64) -> bool {
self.pk_bloom_filter.check_hash(pk_hash)
}
/// Get the schema.
pub fn schema(&self) -> &Arc<ArrowSchema> {
&self.schema
}
/// Get the Lance schema.
pub fn lance_schema(&self) -> &Schema {
&self.lance_schema
}
/// Get the generation number.
pub fn generation(&self) -> u64 {
self.generation
}
/// Get total row count.
pub fn row_count(&self) -> usize {
self.batch_store.total_rows()
}
/// Get batch count.
pub fn batch_count(&self) -> usize {
self.batch_store.len()
}
/// Get batch count (async version for API compatibility).
#[allow(clippy::unused_async)]
pub async fn batch_count_async(&self) -> usize {
self.batch_count()
}
/// Get estimated size in bytes.
pub fn estimated_size(&self) -> usize {
self.batch_store.estimated_bytes() + self.pk_bloom_filter.estimated_memory_size()
}
/// Get the WAL batch mapping.
pub fn wal_batch_mapping(&self) -> &HashMap<usize, (u64, usize)> {
&self.wal_batch_mapping
}
/// Get the last flushed WAL entry position.
pub fn last_flushed_wal_entry_position(&self) -> u64 {
self.last_flushed_wal_entry_position
}
/// Get the bloom filter for serialization.
pub fn bloom_filter(&self) -> &Sbbf {
&self.pk_bloom_filter
}
/// Get reference to indexes.
pub fn indexes(&self) -> Option<&IndexStore> {
self.indexes.as_ref().map(|arc| arc.as_ref())
}
/// Get the Arc-wrapped indexes (for sharing with async handler).
pub fn indexes_arc(&self) -> Option<Arc<IndexStore>> {
self.indexes.clone()
}
/// Take the index registry (for flushing).
/// Returns the Arc, which may be shared with async handler.
pub fn take_indexes(&mut self) -> Option<Arc<IndexStore>> {
self.indexes.take()
}
/// Check if all batches have been flushed to WAL.
pub fn all_flushed_to_wal(&self) -> bool {
self.batch_store.pending_wal_flush_count() == 0
}
/// Get unflushed batch IDs.
pub fn unflushed_batch_positions(&self) -> Vec<usize> {
let batch_count = self.batch_count();
(0..batch_count)
.filter(|id| !self.flushed_batch_positions.contains(id))
.collect()
}
/// Get cache configuration.
pub fn cache_config(&self) -> &CacheConfig {
&self.cache_config
}
/// Get the batch store capacity.
pub fn batch_capacity(&self) -> usize {
self.batch_store.capacity()
}
/// Get remaining batch capacity.
pub fn remaining_batch_capacity(&self) -> usize {
self.batch_store.remaining_capacity()
}
/// Check if batch store is full.
pub fn is_batch_store_full(&self) -> bool {
self.batch_store.is_full()
}
/// Create a scanner for querying this MemTable.
///
/// # Arguments
///
/// * `max_visible_batch_position` - Maximum batch position visible (inclusive)
///
/// The scanner captures the current `max_indexed_batch_position` from the
/// `IndexStore` at construction time to ensure consistent visibility.
///
/// # Panics
///
/// Panics if the memtable has no indexes configured.
///
/// # Example
///
/// ```ignore
/// let scanner = memtable.scan();
/// let results = scanner
/// .project(&["id", "name"])
/// .filter("id > 10")?
/// .try_into_batch()
/// .await?;
/// ```
pub fn scan(&self) -> scanner::MemTableScanner {
let indexes = self
.indexes
.clone()
.expect("MemTable must have indexes configured for scanning");
scanner::MemTableScanner::new(self.batch_store.clone(), indexes, self.schema.clone())
}
/// Get a clone of the batch store Arc for external use.
pub fn batch_store(&self) -> Arc<BatchStore> {
self.batch_store.clone()
}
}
/// Compute a hash for a row's primary key values.
fn compute_row_hash(columns: &[Arc<dyn Array>], row_idx: usize) -> u64 {
use std::hash::{Hash, Hasher};
let mut hasher = std::collections::hash_map::DefaultHasher::new();
for col in columns {
// Hash the scalar value at this row
let is_null = col.is_null(row_idx);
is_null.hash(&mut hasher);
if !is_null {
// Hash based on data type
if let Some(arr) = col.as_any().downcast_ref::<arrow_array::Int32Array>() {
arr.value(row_idx).hash(&mut hasher);
} else if let Some(arr) = col.as_any().downcast_ref::<arrow_array::Int64Array>() {
arr.value(row_idx).hash(&mut hasher);
} else if let Some(arr) = col.as_any().downcast_ref::<arrow_array::StringArray>() {
arr.value(row_idx).hash(&mut hasher);
} else if let Some(arr) = col.as_any().downcast_ref::<arrow_array::BinaryArray>() {
arr.value(row_idx).hash(&mut hasher);
}
// Add more types as needed
}
}
hasher.finish()
}
#[cfg(test)]
mod tests {
use super::*;
use arrow_array::{Int32Array, StringArray};
use arrow_schema::{DataType, Field};
fn create_test_schema() -> Arc<ArrowSchema> {
Arc::new(ArrowSchema::new(vec![
Field::new("id", DataType::Int32, false),
Field::new("name", DataType::Utf8, true),
]))
}
fn create_test_batch(schema: &ArrowSchema, num_rows: usize) -> RecordBatch {
RecordBatch::try_new(
Arc::new(schema.clone()),
vec![
Arc::new(Int32Array::from_iter_values(0..num_rows as i32)),
Arc::new(StringArray::from_iter_values(
(0..num_rows).map(|i| format!("name_{}", i)),
)),
],
)
.unwrap()
}
#[tokio::test]
async fn test_memtable_insert() {
let schema = create_test_schema();
let mut memtable = MemTable::new(schema.clone(), 1, vec![]).unwrap();
let batch = create_test_batch(&schema, 10);
let batch_position = memtable.insert(batch).await.unwrap();
assert_eq!(batch_position, 0);
assert_eq!(memtable.row_count(), 10);
assert_eq!(memtable.batch_count(), 1);
// Dataset is constructed on-demand
assert!(memtable.get_or_create_dataset().await.unwrap().is_some());
}
#[tokio::test]
async fn test_memtable_multiple_inserts() {
let schema = create_test_schema();
let mut memtable = MemTable::new(schema.clone(), 1, vec![]).unwrap();
for i in 0..3 {
let batch = create_test_batch(&schema, 10);
let batch_position = memtable.insert(batch).await.unwrap();
assert_eq!(batch_position, i);
}
assert_eq!(memtable.row_count(), 30);
assert_eq!(memtable.batch_count(), 3);
}
#[tokio::test]
async fn test_memtable_scan() {
let schema = create_test_schema();
let mut memtable = MemTable::new(schema.clone(), 1, vec![]).unwrap();
memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap();
memtable
.insert(create_test_batch(&schema, 5))
.await
.unwrap();
let batches = memtable.scan_batches().await.unwrap();
let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum();
assert_eq!(total_rows, 15);
}
#[tokio::test]
async fn test_memtable_wal_mapping() {
let schema = create_test_schema();
let mut memtable = MemTable::new(schema.clone(), 1, vec![]).unwrap();
let batch_position = memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap();
assert!(!memtable.all_flushed_to_wal());
memtable.mark_wal_flushed(&[batch_position], 5, &[0]);
assert!(memtable.all_flushed_to_wal());
assert_eq!(
memtable.wal_batch_mapping().get(&batch_position),
Some(&(5, 0))
);
assert_eq!(memtable.last_flushed_wal_entry_position(), 5);
}
#[tokio::test]
async fn test_memtable_unflushed_batches() {
let schema = create_test_schema();
let mut memtable = MemTable::new(schema.clone(), 1, vec![]).unwrap();
let batch1 = memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap();
let batch2 = memtable
.insert(create_test_batch(&schema, 5))
.await
.unwrap();
assert_eq!(memtable.unflushed_batch_positions(), vec![batch1, batch2]);
memtable.mark_wal_flushed(&[batch1], 1, &[0]);
assert_eq!(memtable.unflushed_batch_positions(), vec![batch2]);
}
#[tokio::test]
async fn test_memtable_visibility_tracking() {
let schema = create_test_schema();
let mut memtable = MemTable::new(schema.clone(), 1, vec![]).unwrap();
// Insert batches at positions 0, 1, 2
memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap();
memtable
.insert(create_test_batch(&schema, 5))
.await
.unwrap();
memtable
.insert(create_test_batch(&schema, 3))
.await
.unwrap();
// max_visible_batch_position=1 means positions 0 and 1 are visible
let visible = memtable.get_visible_batches(1).await;
assert_eq!(visible.len(), 2);
let total_rows: usize = visible.iter().map(|b| b.num_rows()).sum();
assert_eq!(total_rows, 15); // 10 + 5
// max_visible_batch_position=2 means all batches are visible
let visible = memtable.get_visible_batches(2).await;
assert_eq!(visible.len(), 3);
// max_visible_batch_position=0 means only position 0 is visible
let visible = memtable.get_visible_batches(0).await;
assert_eq!(visible.len(), 1);
}
#[tokio::test]
async fn test_memtable_get_max_visible_batch_positions() {
let schema = create_test_schema();
let mut memtable = MemTable::new(schema.clone(), 1, vec![]).unwrap();
// Insert batches at positions 0, 1, 2
memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap();
memtable
.insert(create_test_batch(&schema, 5))
.await
.unwrap();
memtable
.insert(create_test_batch(&schema, 3))
.await
.unwrap();
// max_visible_batch_position=1 means positions 0 and 1 visible
let visible_ids = memtable.get_max_visible_batch_positions(1).await;
assert_eq!(visible_ids, vec![0, 1]);
// max_visible_batch_position=2 means all positions visible
let visible_ids = memtable.get_max_visible_batch_positions(2).await;
assert_eq!(visible_ids, vec![0, 1, 2]);
// max_visible_batch_position=0 means only position 0 visible
let visible_ids = memtable.get_max_visible_batch_positions(0).await;
assert_eq!(visible_ids, vec![0]);
}
#[tokio::test]
async fn test_memtable_is_batch_visible() {
let schema = create_test_schema();
let mut memtable = MemTable::new(schema.clone(), 1, vec![]).unwrap();
memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap(); // position 0
memtable
.insert(create_test_batch(&schema, 5))
.await
.unwrap(); // position 1
memtable
.insert(create_test_batch(&schema, 3))
.await
.unwrap(); // position 2
// batch_position 0 is visible when max_visible_batch_position >= 0
assert!(memtable.is_batch_visible(0, 0).await);
assert!(memtable.is_batch_visible(0, 1).await);
assert!(memtable.is_batch_visible(0, 2).await);
// batch_position 2 is only visible when max_visible_batch_position >= 2
assert!(!memtable.is_batch_visible(2, 1).await);
assert!(memtable.is_batch_visible(2, 2).await);
assert!(memtable.is_batch_visible(2, 3).await);
// Non-existent batch
assert!(!memtable.is_batch_visible(999, 100).await);
}
#[tokio::test]
async fn test_memtable_scan_batches_at_position() {
let schema = create_test_schema();
let mut memtable = MemTable::new(schema.clone(), 1, vec![]).unwrap();
memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap(); // position 0
memtable
.insert(create_test_batch(&schema, 5))
.await
.unwrap(); // position 1
let batches = memtable.scan_batches_at_position(0).await.unwrap();
assert_eq!(batches.len(), 1);
assert_eq!(batches[0].num_rows(), 10);
let batches = memtable.scan_batches_at_position(1).await.unwrap();
assert_eq!(batches.len(), 2);
}
#[tokio::test]
async fn test_memtable_capacity() {
let schema = create_test_schema();
let mut memtable =
MemTable::with_capacity(schema.clone(), 1, vec![], CacheConfig::default(), 3).unwrap();
assert_eq!(memtable.batch_capacity(), 3);
assert_eq!(memtable.remaining_batch_capacity(), 3);
assert!(!memtable.is_batch_store_full());
// Fill up the store
memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap();
memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap();
memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap();
assert!(memtable.is_batch_store_full());
assert_eq!(memtable.remaining_batch_capacity(), 0);
// Next insert should fail
let result = memtable.insert(create_test_batch(&schema, 10)).await;
assert!(result.is_err());
}
#[tokio::test]
async fn test_memtable_should_flush() {
let schema = create_test_schema();
let mut memtable =
MemTable::with_capacity(schema.clone(), 1, vec![], CacheConfig::default(), 2).unwrap();
// Not full yet
assert!(!memtable.should_flush(1024 * 1024));
memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap();
memtable
.insert(create_test_batch(&schema, 10))
.await
.unwrap();
// Now full
assert!(memtable.should_flush(1024 * 1024));
}
}