diff --git a/docs/src/format/table/.pages b/docs/src/format/table/.pages index ec66d452eb6..69209d956cc 100644 --- a/docs/src/format/table/.pages +++ b/docs/src/format/table/.pages @@ -5,4 +5,5 @@ nav: - Layout: layout.md - Branch & Tag: branch_tag.md - Row ID & Lineage: row_id_lineage.md + - MemTable & WAL: mem_wal.md - index diff --git a/docs/src/format/table/index/system/.pages b/docs/src/format/table/index/system/.pages index cedf138336b..03435c92bf2 100644 --- a/docs/src/format/table/index/system/.pages +++ b/docs/src/format/table/index/system/.pages @@ -1,4 +1,4 @@ title: System Indices nav: - Fragment Reuse: frag_reuse.md - - MemWAL: memwal.md + - MemWAL: mem_wal.md diff --git a/docs/src/format/table/index/system/mem_wal.md b/docs/src/format/table/index/system/mem_wal.md new file mode 100644 index 00000000000..73693cd6704 --- /dev/null +++ b/docs/src/format/table/index/system/mem_wal.md @@ -0,0 +1,12 @@ +# MemWAL Index + +The MemWAL Index is a system index that serves as the centralized structure for all MemWAL metadata. +It stores configuration (region specs, indexes to maintain), merge progress, and region state snapshots. + +A table has at most one MemWAL index. + +For the complete specification, see: + +- [MemWAL Index Overview](../../mem_wal.md#memwal-index) - Purpose and high-level description +- [MemWAL Index Details](../../mem_wal.md#memwal-index-details) - Storage format, schemas, and staleness handling +- [MemWAL Index Builder](../../mem_wal.md#memwal-index-builder) - Background process and configuration updates diff --git a/docs/src/format/table/index/system/memwal.md b/docs/src/format/table/index/system/memwal.md deleted file mode 100644 index 41e2948409c..00000000000 --- a/docs/src/format/table/index/system/memwal.md +++ /dev/null @@ -1,27 +0,0 @@ -# MemWAL Index - -The MemTable and Write-Ahead Log (MemWAL) Index is used for fast upserts into the Lance table. - -The index is used as the centralized synchronization system for a log-structured merge tree (LSM-tree), -leaving the actual implementation of the MemTable and WAL up to the specific implementer of the spec. - -Each region represents a single writer that writes to both a MemTable and a WAL, -and a region can have increasing generations of MemWALs. -Every time data is written into a WAL, the index is updated with the latest watermark. -If a specific writer of a region dies, a new writer is able to read the information in the specific region and replay the WAL. - -## Index Details - -```protobuf -%%% proto.message.MemWalIndexDetails %%% -``` - -## Expected Use Pattern - -It is expected that: - -1. there is exactly one writer for each region, guaranteed by optimistic update of the owner_id -2. each writer updates the MemWAL index after a successful write to WAL and MemTable -3. a new writer always finds unsealed MemWALs and performs replay before accepting new writes -4. background processes are responsible for merging flushed MemWALs to the main Lance table, and making index up to date. -5. a MemWAL-aware reader is able to merge results of MemTables in the MemWALs with results in the base Lance table. \ No newline at end of file diff --git a/docs/src/format/table/mem_wal.md b/docs/src/format/table/mem_wal.md new file mode 100644 index 00000000000..74cf3ab72cb --- /dev/null +++ b/docs/src/format/table/mem_wal.md @@ -0,0 +1,1006 @@ +# MemTable & WAL Specification (Experimental) + +Lance MemTable & WAL (MemWAL) specification describes a Log-Structured-Merge (LSM) tree architecture for Lance tables, enabling high-performance streaming write workloads while maintaining indexed read performance for key workloads including +scan, point lookup, vector search and full-text search. + +!!!note + MemWAL requires the table to have an [unenforced primary key](index.md#unenforced-primary-key) defined. + +## Overall Architecture + +![MemWAL Overview](../../images/mem_wal_overview.png) + +### Base Table + +Under the MemWAL setup, the Lance table is called the **base table**. + +### Region + +A **Region** is the main unit to horizontally scale out writes. + +Each region has exactly one active writer at any time, using **epoch-based fencing** to guarantee single-writer semantics without distributed coordination. +Writers claim a region by incrementing the writer epoch, then write data to that region. +Data in each region is merged into the base table gradually in the background. + +Regions must contain rows that are **mutually exclusive**. +Two regions contain rows with the same primary key, the following scenario can cause data corruption: + +1. Region A receives a write with primary key `pk=1` at time T1 +2. Region B receives a write with primary key `pk=1` at time T2 (T2 > T1) +3. The row in region B is merged into the base table first +4. The row in region A is merged into the base table second +5. The row from Region A (older) now overwrites the row from Region B (newer) + +This violates the expected "last write wins" semantics. +By ensuring each primary key is assigned to exactly one region via the region spec, merge order between regions becomes irrelevant for correctness. + +### MemWAL Index + +A **MemWAL Index** is the centralized structure for all MemWAL metadata for a base table. +A table has at most one MemWAL index. +It stores: + +- **Configuration**: Region specs defining how rows map to regions, and which indexes to maintain +- **Merge progress**: Last generation merged to base table for each region +- **Index catchup progress**: Which merged generation each base table index has been rebuilt to cover +- **Region snapshots**: Point-in-time snapshot of all region states for read optimization + +The index is the source of truth for **configuration** and **merge progress**, but region state snapshots are for read-only optimization (each region's manifest is authoritative for its own state). + +Writers read the MemWAL index to get configuration (region specs, maintained indexes) before writing. +Readers use the index to get a [snapshot of all region states](#reader-consistency), then query each region's data alongside the base table and merge results at runtime. + +A [background process](#memwal-index-builder) periodically updates region snapshots by listing regions and loading their manifests. +See [MemWAL Index Details](#memwal-index-details) for the complete structure. + +## Region Architecture + +![Region Architecture](../../images/mem_wal_regional.png) + +Within a region, writes enter its MemTable and are flushed to the regional WAL for durability. +The MemTable is flushed to storage as a Flushed MemTable based on memory pressure and other conditions. +Flushed MemTables are then asynchronously merged into the base table. + +### MemTable + +An in-memory Lance table that buffers incoming writes. +Each write inserts a fragment in the MemTable, making data immediately queryable without waiting for persistence. + +In addition to the data fragments, a MemTable maintains: + +- **Primary key bloom filter**: For efficient existence checks during staleness detection +- **In-memory index builders**: Incremental index structures that mirror base table indexes, enabling indexed queries on unflushed data +- **WAL fragment mapping**: Tracks correspondence between MemTable fragment IDs and WAL entry IDs for index remapping during flush + +### WAL + +Write-Ahead Log (WAL) serves as the durable storage of MemTable. +A write to MemTable must be persisted also to the WAL to become fully durable. +Every time we write to the WAL, we call it a **WAL Flush**. + +The whole LSM tree's durability is determined by the durability of the WAL. +For example, if WAL is stored in Amazon S3, it has the 99.999999999% durability. +If it is stored in local disk, the data will be lost if the local disk is damaged. + +A WAL consists of an ordered sequence of WAL entries starting from 1. Each entry is a Lance format file. +The writer epoch is stored in the Lance file's schema metadata with key `writer_epoch` for fencing validation during replay. + +Each WAL entry is stored within the WAL directory of the region located at `_mem_wal/{region_id}/wal`. + +WAL files use bit-reversed 64-bit binary naming to distribute files evenly across the directory keyspace. +This optimizes S3 throughput by spreading sequential writes across S3's internal partitions, minimizing throttling. +The filename is the bit-reversed binary representation of the entry ID with suffix `.lance`. +For example, entry ID 5 (binary `000...101`) becomes `1010000000000000000000000000000000000000000000000000000000000000.lance`. + +### Flushed MemTable + +A flushed MemTable is a complete Lance table created by flushing the MemTable to storage. + +!!!note + This is called Sorted String Table (SSTable) or Sorted Run in many LSM-tree literatures and implementations. + However, since our MemTable is not sorted, we just use the term flushed MemTable to avoid confusion. + +Each flushed MemTable has a **generation** number starting from 1 that identifies its relative position among all flushed MemTables in the region. +When MemTable with generation `i` is flushed, the next MemTable gets generation number `i+1`. + +The MemTable of generation `i` is flushed to `_mem_wal/{region_uuid}/{random_hash}_gen_{i}/` directory, where `{random_hash}` is an 8-character hex value generated at flush time. +The directory content follows [Lance table layout](layout.md). + +The actual directory path for each generation is recorded in the region manifest's `flushed_generations` list (see [Region Manifest](#region-manifest)). + +Generation numbers determine merge order: lower numbers represent older data and must be merged to the base table first to preserve correct upsert semantics. + +### Region Manifest + +Each region has a manifest file containing epoch-based fencing tokens, WAL pointers, and flushed MemTable generation trackers. This is the source of truth for region state. + +The manifest is serialized as a protobuf binary file using the `RegionManifest` message. + +The manifest contains: + +- **Fencing state**: `writer_epoch` (writer fencing token) +- **WAL pointers**: `replay_after_wal_id` (last entry flushed to MemTable), `wal_id_last_seen` (last entry seen at manifest update) +- **Generation trackers**: `current_generation` (next generation to flush) +- **Flushed generations**: `flushed_generations` list of generation number and directory path pairs (e.g., generation 1 at `a1b2c3d4_gen_1`) + +Note: `wal_id_last_seen` is a hint that may be stale since it's not updated on WAL write. +The manifest itself is atomically written, but recovery must try to get newer WAL files to find the actual state beyond this hint. + +
+RegionManifest protobuf message + +```protobuf +%%% mem_wal.message.RegionManifest %%% +``` + +
+ +Manifests are versioned starting from 1 and immutable. Each update creates a new manifest file at the next version number. +Updates use put-if-not-exists or file rename to ensure atomicity depending on the storage system. +If two processes compete, one wins and the other retries. + +To commit a manifest version: + +1. Compute the next version number +2. Write the manifest to `{bit_reversed_version}.binpb` using put-if-not-exists +3. In parallel best-effort write to `version_hint.json` with `{"version": }` (failure is acceptable) + +To read the latest manifest version: + +1. Read `version_hint.json` to get the latest version hint. If not found, start from version 1 +2. Check existence for subsequent versions from the starting version +3. Continue until a version is not found +4. The latest version is the last found version + +This approach uses HEAD requests instead of LIST operations in cloud storage, which is generally faster and is friendly to systems like S3 Express that do not support lexicographically sorted listing. + +!!!note + This works because the write rate to region manifests is significantly lower than read rates. Region manifests are only updated when region metadata changes (MemTable flush), not on every write. This ensures HEAD requests will eventually terminate and find the latest version. + +All region manifest versions are stored in `_mem_wal/{region_id}/manifest` directory. + +Each region manifest version file uses bit-reversed 64-bit binary naming, the same scheme as WAL files. +For example, version 5 becomes `1010000000000000000000000000000000000000000000000000000000000000.binpb`. + +The region manifest is updated atomically in the following cases: + +| Trigger | Fields Updated | Details | +|---------|----------------|---------| +| [Initialization & Recovery](#initialization--recovery) | `writer_epoch` | Incremented when writer claims the region | +| [MemTable Flush](#memtable-flush) | `replay_after_wal_id`, `wal_id_last_seen`, `current_generation`, `flushed_generations` | After flushing MemTable to storage | +| [MemWAL Index Builder](#memwal-index-builder) | `wal_id_last_seen` | Periodically scans WAL entries and updates hint | +| [Garbage Collector](#garbage-collector) | `flushed_generations` | Removes entries for deleted flushed MemTables | + +!!!note + WAL flush does **not** update the manifest to keep the hot write path fast. + +Writers use epoch-based fencing (`writer_epoch`) to ensure single-writer semantics. +See [Writer Fencing](#writer-fencing) for details. + +## MemWAL Index Details + +The MemWAL Index uses the [standard index storage](index/index.md#index-storage) at `_indices/{UUID}/`. + +The index stores its data in two parts: + +1. **Index details** (`index_details` in `IndexMetadata`): Contains configuration, merge progress, and snapshot metadata +2. **Region snapshots**: Stored as a Lance file or inline, depending on region count + +### Index Details + +The `index_details` field in `IndexMetadata` contains a `MemWalIndexDetails` protobuf message with the following key fields: + +**Configuration fields** (`region_specs`, `maintained_indexes`) are the source of truth for MemWAL configuration. +Writers read these fields to determine how to partition data and which indexes to maintain. + +**Merge progress** (`merged_generations`) tracks the last generation merged to the base table for each region. +This field is updated atomically with merge-insert data commits, enabling conflict resolution when multiple mergers operate concurrently. +Each entry contains the region UUID and generation number. + +**Index catchup progress** (`index_catchup`) tracks which merged generation each base table index has been rebuilt to cover. +When data is merged from a flushed MemTable to the base table, the base table's indexes are rebuilt asynchronously. +During this window, queries should use the flushed MemTable's pre-built indexes instead of scanning unindexed data in the base table. +See [Index Catchup and Read Path](#index-catchup-and-read-path) for details. + +**Region snapshot fields** (`snapshot_ts_millis`, `num_regions`, `inline_snapshots`) provide a point-in-time snapshot of region states. +The actual region manifests remain authoritative for region state. +When `num_regions` is 0, the `inline_snapshots` field may be `None` or an empty Lance file with 0 rows but proper schema. + +
+MemWalIndexDetails protobuf message + +```protobuf +%%% mem_wal.message.MemWalIndexDetails %%% +``` + +
+ +### Region Identifier + +Each region has a unique identifier across all regions following UUID v4 standard. +When a new region is created, it is assigned a new identifier. + +### Region Spec + +A **Region Spec** defines how all rows in a table are logically divided into different regions, +enabling automatic region assignment and query-time region pruning. + +Each region spec has: + +- **Spec ID**: A positive integer that uniquely identifies this spec within the MemWAL index. IDs are never reused. +- **Region fields**: An array of field definitions that determine how to compute region values. + +Each region is bound to a specific region spec ID, recorded in its [manifest](#region-manifest). +Regions without a spec ID (`spec_id = 0`) are manually-created regions not governed by any spec. + +A region spec's field array consists of **region field** definitions. +Each region field has the following properties: + +| Property | Description | +|----------|-------------| +| `field_id` | Unique string identifier for this region field | +| `source_ids` | Array of field IDs referencing source columns in the schema | +| `transform` | A well-known region expression, specify this or `expression` | +| `expression` | A DataFusion SQL expression for custom logic, specify this or `transform` | +| `result_type` | The output type of the region value | + +#### Region Expression + +A **Region Expression** is a [DataFusion SQL expression](https://datafusion.apache.org/user-guide/sql/index.html) that derives a region value from source column(s). +Source columns are referenced as `col0`, `col1`, etc., corresponding to the order of field IDs in `source_ids`. + +Region expressions must satisfy the following requirements: + +1. **Deterministic**: The same input value must always produce the same output value. +2. **Stateless**: The expression must not depend on external state (e.g., current time, random values, session variables). +3. **Type-promotion resistant**: The expression must produce the same result for equivalent values regardless of their numeric type (e.g., `int32(5)` and `int64(5)` must yield the same region value). +4. **Column removal resistant**: If a source field ID is not found in the schema, the column should be interpreted as NULL. +5. **NULL-safe**: The expression should properly handle NULL inputs and have defined behavior (e.g., return NULL if input is NULL for single-column expressions). +6. **Consistent with result type**: The expression's return type must be consistent with `result_type` in non-NULL cases. + +#### Region Transform + +A **Region Transform** is a well-known region expression with a predefined name. +When a transform is specified, the expression is derived automatically. + +| Transform | Parameters | Region Expression | Result Type | +|-----------|------------|-------------------|-------------| +| `identity` | (none) | `col0` | same as source | +| `year` | (none) | `date_part('year', col0)` | `int32` | +| `month` | (none) | `date_part('month', col0)` | `int32` | +| `day` | (none) | `date_part('day', col0)` | `int32` | +| `hour` | (none) | `date_part('hour', col0)` | `int32` | +| `bucket` | `num_buckets` | `abs(murmur3(col0)) % N` | `int32` | +| `multi_bucket` | `num_buckets` | `abs(murmur3_multi(col0, col1, ...)) % N` | `int32` | +| `truncate` | `width` | `left(col0, W)` (string) or `col0 - (col0 % W)` (numeric) | same as source | + +The `bucket` and `multi_bucket` transforms use Murmur3 hash functions: + +- **`murmur3(col)`**: Computes the 32-bit Murmur3 hash (x86 variant, seed 0) of a single column. Returns a signed 32-bit integer. Returns NULL if input is NULL. +- **`murmur3_multi(col0, col1, ...)`**: Computes the Murmur3 hash across multiple columns. Returns a signed 32-bit integer. NULL fields are ignored during hashing; returns NULL only if all inputs are NULL. + +The hash result is wrapped with `abs()` and modulo `N` to produce a non-negative bucket number in the range `[0, N)`. + +### Region Snapshot Storage + +Region snapshots are stored using one of two strategies based on the number of regions: + +| Region Count | Storage Strategy | Location | +|--------------|------------------|----------| +| <= 100 (threshold) | Inline | `inline_snapshots` field in index details | +| > 100 | External Lance file | `_indices/{UUID}/index.lance` | + +The threshold (100 regions) is implementation-defined and may vary. + +**Inline storage**: For small region counts, snapshots are serialized as a Lance file and stored in the `inline_snapshots` field. +This keeps the index metadata compact while avoiding an additional file read for common cases. + +**External Lance file**: For large region counts, snapshots are stored as a Lance file at `_indices/{UUID}/index.lance`. +This file uses standard Lance format with the region snapshot schema, enabling efficient columnar access and compression. + +### Region Snapshot Arrow Schema + +Region snapshots are stored as a Lance file with one row per region. +The schema has one column per `RegionManifest` field plus region spec columns: + +| Column | Type | Description | +|--------|------|-------------| +| `region_id` | `fixed_size_binary(16)` | Region UUID bytes | +| `version` | `uint64` | Region manifest version | +| `region_spec_id` | `uint32` | Region spec ID (0 if manual) | +| `writer_epoch` | `uint64` | Writer fencing token | +| `replay_after_wal_id` | `uint64` | Last WAL entry flushed to MemTable | +| `wal_id_last_seen` | `uint64` | Last WAL entry seen (hint) | +| `current_generation` | `uint64` | Next generation to flush | +| `flushed_generations` | `list>` | Flushed MemTable paths | +| `region_field_{field_id}` | varies | Region field value (one column per field in region spec) | + +For example, with a region spec containing a field `user_bucket` of type `int32`: + +| Column | Type | Description | +|--------|------|-------------| +| ... | ... | (base columns above) | +| `region_field_user_bucket` | `int32` | Bucket value for this region | + +This schema directly corresponds to the fields in the `RegionManifest` protobuf message plus the computed region field values. + +### Vector Index Configuration + +If the main use case is IVF family vector index, it is recommended to have these indexes on the Lance table before enabling MemWAL. +This is because IVF index needs to remain the same quantization codebook (e.g. PQ codebook) across all the layers of the LSM tree for vector distance to be comparable. + +MemTables automatically inherit vector indexing from base table indexes. +For each vector index on the base table, MemTable uses the same index type (IVF-PQ, IVF-SQ, etc.) with the same centroids and quantization parameters. +This ensures distances are precise and comparable across generations. + +The base table vector index should not change the codebook once MemWAL is enabled. +To switch codebooks, a migration is required: create another vector index with the new codebook, configure MemTable to maintain both indexes, and eventually drop the old index after all readers are using the new codebook and all MemTables have indexes using the new codebook. + +## Storage Layout + +Here is a recap of the storage layout with all the files and concepts defined so far: + +``` +{table_path}/ +├── _indices/ +│ └── {index_uuid}/ # MemWAL Index (uses standard index storage) +│ └── index.lance # Serialized region snapshots (Lance file) +│ +└── _mem_wal/ + └── {region_uuid}/ # Region directory (UUID v4) + ├── manifest/ + │ ├── {bit_reversed_version}.binpb # Serialized region manifest (bit-reversed naming) + │ └── version_hint.json # Version hint file + ├── wal/ + │ ├── {bit_reversed_entry_id}.lance # WAL data files (bit-reversed naming) + │ └── ... + └── {random_hash}_gen_{i}/ # Flushed MemTable (generation i, random prefix) + ├── _versions/ + │ └── {version}.manifest # Table manifest (V2 naming scheme) + ├── _indices/ # Indexes + │ ├── {vector_index}/ + │ └── {scalar_index}/ + └── bloom_filter.bin # Primary key bloom filter +``` + +## Writer Expectations + +A writer operates on a single region within a single process and may spawn asynchronous tasks for background operations like WAL flush and MemTable flush. + +### Writer Configuration + +Writers can be configured with the following options that affect write behavior: + +| Option | Description | +|--------|-------------| +| **Durable write** | Each write is persisted to WAL before reporting success. Ensures no data loss on crash, but adds latency for object storage writes. | +| **Indexed write** | Each write refreshes MemTable indexes before reporting success. Ensures new data is immediately searchable via indexes, but adds indexing latency. | + +Both options can be enabled independently. When disabled: + +- **Non-durable writes** buffer data in memory until a flush threshold is reached, accepting potential data loss on crash +- **Non-indexed writes** defer index updates, meaning newly written data may not appear in index-accelerated queries until the next index refresh + +### Synchronous vs Asynchronous Operations + +Writer operations can be categorized by their synchronous or asynchronous nature: + +| Operation | Mode | Description | +|-----------|------|-------------| +| [Initialization & Recovery](#initialization--recovery) | Synchronous | Claims region and replays WAL entries | +| [Write to MemTable](#write-operations) | Synchronous | Data inserted into in-memory fragments | +| [WAL Flush](#wal-flush) | Configurable | Synchronous with durable writes, asynchronous otherwise | +| [Index Update](#memtable-indexing) | Configurable | Synchronous with indexed writes, asynchronous otherwise | +| [MemTable Flush](#memtable-flush) | Asynchronous | Triggered by thresholds, runs in background | + +### Initialization & Recovery + +A writer must claim a region before performing any write operations: + +1. Load the latest region manifest +2. Increment `writer_epoch` by one +3. Atomically write a new manifest +4. If the write fails (another writer claimed the epoch), reload the manifest and retry with a higher epoch +5. After initialization, read WAL entries sequentially from `replay_after_wal_id + 1` until not found +6. Replay valid WAL entries (those with `writer_epoch` <= current epoch) to reconstruct the MemTable with 1:1 [WAL fragment mapping](#wal-fragment-mapping-construction) (each WAL entry becomes one MemTable fragment) + +After initialization, the writer updates the [WAL fragment mapping](#wal-fragment-mapping-construction) as new [WAL flushes](#wal-flush) occur. + +### Write Operations + +Each write operation follows this sequence: + +1. Validate incoming records +2. Insert records into the MemTable, creating an in-memory fragment (immediately queryable via full scan) +3. Track the Lance data file in the new fragment for pending WAL flush +4. Optionally trigger WAL flush based on size, count, or time thresholds +5. For [durable writes](#writer-configuration), wait for WAL flush to complete before returning +6. For [indexed writes](#writer-configuration), update MemTable indexes before returning: + - Insert primary keys into the bloom filter + - For each vector column with a base table index: encode and insert using the same index type as base table + - For each index in `maintained_indexes`: update the corresponding index structure + +### WAL Flush + +WAL flush batches pending MemTable fragments into a single Lance data file: + +1. Identify pending (unflushed) fragments in the MemTable +2. Start writing the WAL entry to object storage +3. Stream binary pages from each pending fragment's Lance data file directly to the WAL entry +4. Write the footer containing batched data file metadata and `writer_epoch` in schema metadata +5. Complete the WAL entry write atomically +6. Mark fragments as flushed in the MemTable +7. Update the [WAL fragment mapping](#wal-fragment-mapping-construction) (MemTable fragment IDs in this batch -> WAL entry ID and positions) for index remapping during [MemTable Flush](#memtable-flush) + +!!!note + The region manifest is **not** updated on every WAL flush. The `wal_id_last_seen` field is a hint that can be updated: + + 1. **During MemTable flush** - when the region manifest is updated anyway + 2. **By a background index builder** - which scans WAL entries and updates each region's `wal_id_last_seen` + + This keeps the hot write path fast. On recovery, the writer reads WAL entries sequentially starting from `wal_id_last_seen + 1` to discover any WAL entries beyond what the manifest indicates. + +The WAL flush behavior depends on the [durable write](#writer-configuration) option: + +| Mode | Behavior | Result | +|------|----------|--------| +| Durable write | Flush immediately, wait for completion | One or more Lance files per write | +| Non-durable write | Buffer until threshold, return immediately | Batched Lance files (fewer S3 operations) | + +### MemTable Indexing + +MemTable indexing differs from base table indexing to balance write performance with query capability. +Rather than maintaining all base table indexes, MemTables maintain a subset specified in the [MemWAL Index](#memwal-index). + +MemTables maintain a **primary key bloom filter** for efficiently checking whether a primary key exists in a generation. +This enables staleness detection during search queries without requiring expensive point lookups. + +For vector indexes, MemTables use the same index type as the base table (e.g., IVF-PQ with the same centroids and PQ codebook). +This ensures distances are precise and directly comparable across generations. +The centroid assignment also impacts recall, so using the same centroids ensures consistent search quality. + +For full-text search indexes, MemTables inherit tokenizer configuration from base table indexes to ensure consistent tokenization across generations. +Each generation maintains its own corpus statistics (document count, term frequencies) which are aggregated at query time for globally-comparable BM25 scores. + +When a MemTable is flushed to storage: + +1. Indexes are serialized to disk in the flushed MemTable's `_indices/` directory following the Lance table index format +2. The primary key bloom filter is serialized to `bloom_filter.bin` in the generation directory +3. The in-memory index structures may be retained as a cache for readers in the same process + +### WAL Fragment Mapping Construction + +The WAL fragment mapping tracks the correspondence between MemTable fragment IDs and WAL entry IDs. +This mapping is essential for remapping indexes during [MemTable flush](#memtable-flush), since indexes reference MemTable fragment IDs but the flushed MemTable references WAL entry IDs. + +The mapping is structured as: `MemTable fragment ID -> (WAL entry ID, position within entry)` + +Where: + +- **MemTable fragment ID**: The fragment's position in the MemTable (0-indexed within the current generation) +- **WAL entry ID**: The WAL entry containing this fragment's data (relative to `replay_after_wal_id`) +- **Position within entry**: The fragment's position within the WAL entry (since multiple fragments may be batched) + +The mapping is updated in two scenarios: + +1. **[Initialization & Recovery](#initialization--recovery)**: During WAL replay, each replayed WAL entry creates MemTable fragments with 1:1 mapping (one fragment per WAL entry, position 0) +2. **[WAL Flush](#wal-flush)**: After flushing pending fragments to a new WAL entry, the mapping records which MemTable fragments were written to which WAL entry and their positions + +During [MemTable flush](#memtable-flush), indexes are remapped by translating MemTable fragment IDs to the corresponding WAL entry references using this mapping. + +### MemTable Flush + +Flushing the MemTable creates a new flushed MemTable (generation) with data and indexes: + +1. Generate a random 8-character hex prefix (e.g., `a1b2c3d4`) +2. Create directory `_mem_wal/{region_uuid}/{random_hash}_gen_{current_generation}/` +3. Identify WAL entries to include (from `replay_after_wal_id + 1` to the last flushed entry) +4. Create table manifest with `base_paths` pointing to the WAL directory +5. Add fragment entries referencing WAL files via `base_id` +6. Remap indexes using the [WAL fragment mapping](#wal-fragment-mapping-construction): + - Read index entries referencing MemTable fragment IDs + - Translate to flushed MemTable fragment IDs using the mapping + - Write remapped indexes to `_mem_wal/{region_uuid}/{random_hash}_gen_{current_generation}/_indices/` +7. Write the manifest to `_mem_wal/{region_uuid}/{random_hash}_gen_{current_generation}/_versions/{version}.manifest` (using [V2 naming scheme](transaction.md#manifest-naming-schemes)) +8. Update the region manifest: + - Advance `replay_after_wal_id` to the last flushed WAL entry + - Update `wal_id_last_seen` + - Increment `current_generation` + - Append `(current_generation, {random_hash}_gen_{current_generation})` to `flushed_generations` + +The random prefix ensures that flush retries write to a new directory, avoiding conflicts with partially written files from failed attempts. Only the directory recorded in `flushed_generations` is considered valid. + +If the writer crashes before completing MemTable flush, the new writer replays WAL entries into memory with 1:1 [WAL fragment mapping](#wal-fragment-mapping-construction), rebuilds the in-memory indexes, and can then perform a fresh MemTable flush with a new random prefix. + +### Writer Fencing + +Before any manifest update (MemTable flush), a writer must verify its `writer_epoch` remains valid: + +- If `local_writer_epoch == stored_writer_epoch`: The writer is still active and may proceed +- If `local_writer_epoch < stored_writer_epoch`: The writer has been fenced and must abort + +Fenced writers must stop all operations immediately and notify pending writes of the failure. + +For a concrete example of fencing between two writers, see [Appendix 1: Writer Fencing Example](#appendix-1-writer-fencing-example). + +## Background Job Expectations + +Background jobs run independently from writers and handle asynchronous maintenance tasks. + +### MemTable Merger + +Flushed MemTables are merged to the base table in generation order using Lance's merge-insert operation. + +#### Merge Workflow + +1. Read `merged_generations[region_id]` +2. Load the region manifest and identify unmerged flushed MemTables from `flushed_generations`: those with generation numbers > `merged_generations[region_id]` +3. For each flushed MemTable in ascending generation order: + - Look up the directory path from `flushed_generations` + - Open it as a Lance table + - Execute merge-insert into the base table, atomically updating the MemWAL Index: + - Set `merged_generations[region_id]` to this generation + - On commit conflict, apply [conflict resolution rules](#conflict-resolution-and-concurrency) +4. After merge, the flushed MemTable and its referenced WAL files may be garbage collected (see [Garbage Collector](#garbage-collector)) + +Ordered merge ensures correct upsert semantics: flushed MemTables with higher generation numbers overwrite those with lower numbers. + +#### Conflict Resolution and Concurrency + +Multiple mergers may operate on the same region concurrently. This is safe due to: + +1. **Atomic update**: `merged_generations` is updated atomically with the data commit +2. **Conflict resolution**: When a merge-insert commit encounters a version conflict, the merger reads the conflicting commit's `merged_generations`. If `merged_generations[region_id] >= my_generation`, abort without retry (data already merged or superseded). Otherwise, retry the commit as normal. +3. **Merge-insert idempotency**: If two mergers merge the same generation before either commits, both write identical data (primary key upsert semantics) + +After aborting due to a conflict, reload the MemWAL Index and region manifest, then continue to the next unmerged generation. + +`merged_generations` is the single source of truth for merge progress. +If a merger crashes after committing, the next merger reads the MemWAL Index to determine which generations are already merged. + +For a concrete example, see [Appendix 2: Concurrent Merger Example](#appendix-2-concurrent-merger-example). + +### MemWAL Index Builder + +A background process periodically builds a new region snapshot: + +1. Load the existing MemWAL Index to preserve configuration (`region_specs`, `maintained_indexes`) and merge progress (`merged_generations`) +2. List all region directories under `_mem_wal/` +3. For each region: + - Load the region manifest + - Scan WAL entries sequentially to find the actual last entry ID + - If the observed WAL ID is greater than `wal_id_last_seen`, update the region manifest (ignore errors since this is best-effort) + - Copy manifest fields (including `flushed_generations`) into a region snapshot row +4. Determine storage strategy based on region count: + - If `num_regions <= threshold`: Serialize as Lance file bytes to `inline_snapshots` + - If `num_regions > threshold`: Write as Lance file to `_indices/{UUID}/index.lance` +5. Create new `MemWalIndexDetails` with preserved configuration, merge progress, and new region snapshots +6. Update the table manifest with the new index metadata + +This process serves two purposes: + +- Keeps `wal_id_last_seen` up-to-date in region manifests (since writers don't update it on every WAL flush) +- Provides readers with an efficient snapshot of all region states + +The build frequency is implementation-defined. More frequent builds reduce staleness but increase I/O overhead. + +### Base Table Index Builder + +A background process rebuilds base table indexes to cover newly merged data and updates `index_catchup` progress in the MemWAL Index. +Typically there is a dedicated builder for each index. + +The index builder workflow is expected to be: +1. Rebuild the base table index to the latest state, this automatically covers all merged generations +2. Read the current `merged_generations` +3. Update the MemWAL Index atomically: + - Set `index_catchup[index_name].caught_up_generations` to match `merged_generations` +4. On commit conflict, reload the MemWAL Index and retry + +### Garbage Collector + +The garbage collector removes obsolete data from the region directory and updates the region manifest to remove entries from `flushed_generations` for deleted flushed MemTables. + +Eligible for deletion: + +1. **Flushed MemTable directories**: Generation directories where `generation <= merged_generations[region_id]` AND `generation <= min(index_catchup[I].caught_up_generation)` for all maintained indexes +2. **WAL data files**: Files referenced only by deleted generations +3. **Old region manifest versions**: Versions older than the current version minus a retention threshold +4. **Orphaned directories**: Directories matching `*_gen_*` pattern but not in `flushed_generations` (from failed flush attempts) + +**Index catchup consideration**: Flushed MemTables must be retained until all base table indexes have caught up. +Since flushed MemTables contain pre-built indexes, they are used for indexed queries when the base table index has not yet been rebuilt to cover the merged data. +Only after all indexes in `maintained_indexes` have `caught_up_generation >= generation` can a flushed MemTable be safely deleted. + +**Time travel consideration**: Garbage collection must not remove generations that are reachable by any retained base table version. When a reader opens an older table version, the MemWAL Index snapshot from that version references specific `merged_generations` values. Generations that satisfy `generation > merged_generations[region_id]` for any retained table version must be preserved. + +Garbage collection must verify that no flushed MemTable still references a WAL file before deletion. + +## Reader Expectations + +### LSM Tree Merging Read + +Readers **MUST** merge results from multiple data sources (base table, flushed MemTables, in-memory MemTables) by primary key to ensure correctness. + +When the same primary key exists in multiple sources, the reader must keep only the newest version based on: + +1. **Generation number** (`_gen`): Higher generation wins. The base table has generation -1, MemTables have positive integers starting from 1. +2. **Row address** (`_rowaddr`): Within the same generation, higher row address wins (later writes within a batch overwrite earlier ones). + +The ordering for "newest" is: highest `_gen` first, then highest `_rowaddr`. + +This deduplication is essential because: + +- A row updated in a MemTable also exists (with older data) in the base table +- A flushed MemTable that has been merged to the base table may not yet be garbage collected, causing the same row to appear in both +- A single write batch may contain multiple updates to the same primary key + +Without proper merging, queries would return duplicate or stale rows. + +### Reader Consistency + +Reader consistency depends on two factors: + +1. access to in-memory MemTables +2. the source of region metadata (either through MemWAL index or region manifests) + +Strong consistency requires access to in-memory MemTables for all regions involved in the query and reading region manifests directly. +Otherwise, the query is eventually consistent due to missing unflushed data or stale MemWAL Index snapshots. + +!!!note + Reading a stale MemWAL Index does not impact correctness, only freshness: + + - **Merged MemTable still in index**: If a flushed MemTable has been merged to the base table but still shows in the MemWAL index, readers query both. This results in some inefficiency for querying the same data twice, but [LSM-tree merging](#lsm-tree-merging-read) ensures correct results since both contain the same data. The inefficiency is also compensated by the fact that the data is covered by index and we rarely end up scanning both data. + - **Garbage collected MemTable still in index**: If a flushed MemTable has been garbage collected, but is still in the MemWAL index, readers would fail to open it and skip it. This is also safe because if it is garbage collected, the data must already exist in the base table. + - **Newly flushed MemTable not in index**: If a newly flushed MemTable is added after the snapshot was built, it is not queried. The result is eventually consistent but correct for the snapshot's point in time. + +### Query Planning + +#### MemTable Collection + +The query planner collects datasets from multiple sources and assembles them for unified query execution. +Datasets come from: + +1. base table (representing already-merged data) +2. flushed MemTables (persisted but not yet merged) +3. optionally in-memory MemTables (if accessible). + +Each dataset is tagged with a generation number: -1 for the base table, and positive integers for MemTable generations. +Within a region, the generation number determines data freshness, with higher numbers representing newer data. +Rows from different regions do not need deduplication since each primary key maps to exactly one region. + +The planner also collects bloom filters from each generation for staleness detection during search queries. + +#### Region Pruning + +Before executing queries, if region spec is available, +the planner evaluates filter predicates against region specs to determine which regions may contain matching data. +This pruning step reduces the number of regions to scan. + +For each filter predicate: + +1. Extract predicates on columns used in region specs +2. Evaluate which region values can satisfy the predicate +3. Prune regions whose values cannot match + +For example, with a region spec using `bucket(user_id, 10)` and a filter `user_id = 123`: + +1. Compute `bucket(123, 10) = 3` +2. Only scan regions with bucket value 3 +3. Skip all other regions + +Region pruning applies to both scan queries and prefilters in search queries. + +#### Indexed Read Plan + +When data is merged from a flushed MemTable to the base table, the base table's indexes are rebuilt asynchronously by the [base table index builders](#base-table-index-builder). +During this window, the merged data exists in the base table but is not yet covered by the base table's indexes. + +Without special handling, indexed queries would fall back to expensive full scans for the unindexed part of the base table. +To maintain indexed read performance, the query planner should use `index_catchup` progress to determine the optimal data source for each query. + +The key insight is that flushed MemTables serve as a bridge between the base table's index catchup and the current merged state. +For a query that requires a specific index for acceleration, when `index_gen < merged_gen`, +the generations in the gap `(index_gen, merged_gen]` have data already merged in the base table but are not covered by the base table's index. +Since flushed MemTables contain pre-built indexes (created during [MemTable flush](#memtable-flush)), queries can use these indexes instead of scanning unindexed data in the base table. +This ensures all reads remain indexed regardless of how far behind the async index builder is. + +See [Appendix 4: Index Catchup Example](#appendix-4-index-catchup-example) for a detailed timeline showing how this works in practice. + +### Query Execution + +Query execution unions datasets within each region and deduplicates by primary key according to [LSM tree merge read](#lsm-tree-merge-read). + +The next few subsections go through the query plan expectations using custom execution nodes optimized for MemWAL's data model. + +All query plans assume the following MemWAL setup: + +``` +base_table: shared across all regions (gen -1) + +region_A: + gen 1: flushed_gen_1 + gen 2: in_memory_memtable + +region_B: + gen 1: flushed_gen_1 + gen 2: flushed_gen_2 + gen 3: in_memory_memtable +``` + +Existing Lance index optimizations (scalar indexes, fragment pruning, etc.) continue to apply within each scan and is omitted. +See [Appendix 3: Execution Nodes](#appendix-3-execution-nodes) for uncommon execution nodes we use here for optimized performance. + +#### Scan Queries + +For scan queries, the base table is scanned once and each region's MemTables are scanned separately. +Deduplication happens per primary key across all generations. + +``` +DeduplicateExec: partition_by=[pk], order_by=[_gen DESC, _rowaddr DESC] + UnionExec + # Base table (shared) + ScanExec: base_table[gen=-1], filter=[pushed_down] + # Region A MemTables + ScanExec: region_A[gen=2], filter=[pushed_down] + ScanExec: region_A[gen=1], filter=[pushed_down] + # Region B MemTables + ScanExec: region_B[gen=3], filter=[pushed_down] + ScanExec: region_B[gen=2], filter=[pushed_down] + ScanExec: region_B[gen=1], filter=[pushed_down] +``` + +#### Point Lookups + +Primary key-based point lookups first determine the target region using the region spec, then short-circuit by checking newest generations first within that region, falling back to the base table. + +Bloom filters optimize point lookups by skipping generations that definitely don't contain the key: + +1. Check the bloom filter for each MemTable generation (newest first) +2. If the bloom filter returns negative, skip that generation (key definitely not present) +3. If the bloom filter returns positive, try to take last matching row of that generation +4. If the key is found, return immediately without checking older generations + +``` +# After region pruning: only region_A needs to be checked +# Bloom filters checked before each scan to skip unnecessary I/O +CoalesceFirstExec: return_first_non_null + BloomFilterGuardExec: bf[region_A][gen=2] + TakeLastExec: region_A[gen=2], filter=[pk = target] + BloomFilterGuardExec: bf[region_A][gen=1] + TakeLastExec: region_A[gen=1], filter=[pk = target] + TakeLastExec: base_table[gen=-1], filter=[pk = target] +``` + +#### Vector Search Queries + +Vector search uses bloom filters to detect stale results across all generations. + +``` +GlobalLimitExec: limit=k + SortExec: order_by=[_dist ASC] + FilterStaleExec: bloom_filters=[bf[region_A][gen=2], bf[region_A][gen=1], bf[region_B][gen=3], bf[region_B][gen=2], bf[region_B][gen=1]] + UnionExec + # Base table (shared) + KNNExec: base_table[gen=-1], k=k + # Region A MemTables + KNNExec: region_A[gen=2], k=k + KNNExec: region_A[gen=1], k=k + # Region B MemTables + KNNExec: region_B[gen=3], k=k + KNNExec: region_B[gen=2], k=k + KNNExec: region_B[gen=1], k=k +``` + +For each candidate from generation G, `FilterStaleExec` checks if the primary key exists in bloom filters of generations > G. +If found, the candidate is filtered out because a newer version exists that was not as relevant to the query. + +#### Full-Text Search Queries + +Full-text search aggregates corpus statistics across all generations for globally-comparable BM25 scores. + +``` +GlobalLimitExec: limit=k + SortExec: order_by=[_bm25 DESC] + FilterStaleExec: bloom_filters=[bf[region_A][gen=2], bf[region_A][gen=1], bf[region_B][gen=3], bf[region_B][gen=2], bf[region_B][gen=1]] + GlobalBM25Exec # Aggregates stats across all generations + UnionExec + # Base table (shared) + FTSExec: base_table[gen=-1], query="search terms" + # Region A MemTables + FTSExec: region_A[gen=2], query="search terms" + FTSExec: region_A[gen=1], query="search terms" + # Region B MemTables + FTSExec: region_B[gen=3], query="search terms" + FTSExec: region_B[gen=2], query="search terms" + FTSExec: region_B[gen=1], query="search terms" +``` + +`GlobalBM25Exec` collects document counts and term frequencies from all FTS indexes, computes global BM25 parameters, and passes them to each `FTSExec` for comparable scoring. + +## Appendices + +### Appendix 1: Writer Fencing Example + +This example demonstrates how epoch-based fencing prevents data corruption when two writers compete for the same region. + +#### Initial State + +``` +Region manifest (version 1): + writer_epoch: 5 + replay_after_wal_id: 10 + wal_id_last_seen: 12 +``` + +#### Scenario + +| Step | Writer A | Writer B | Manifest State | +|------|----------|----------|----------------| +| 1 | Loads manifest, sees epoch=5 | | epoch=5, version=1 | +| 2 | Increments to epoch=6, writes manifest v2 | | epoch=6, version=2 | +| 3 | Starts writing WAL entries 13, 14, 15 | | | +| 4 | | Loads manifest v2, sees epoch=6 | epoch=6, version=2 | +| 5 | | Increments to epoch=7, writes manifest v3 | epoch=7, version=3 | +| 6 | | Starts writing WAL entries 16, 17 | | +| 7 | Tries to flush MemTable, loads manifest | | | +| 8 | Sees epoch=7, but local epoch=6 | | | +| 9 | **Writer A is fenced!** Aborts all operations | | | +| 10 | | Continues writing normally | epoch=7, version=3 | + +#### What Happens to Writer A's WAL Entries? + +Writer A wrote WAL entries 13, 14, 15 with `writer_epoch=6` in their schema metadata. + +When Writer B performs crash recovery or MemTable flush: + +1. Reads WAL entries sequentially starting from `replay_after_wal_id + 1` (entry 13) +2. For each entry, checks existence using HEAD request on the bit-reversed filename +3. Continues until an entry is not found (e.g., entry 18 doesn't exist) +4. Finds entries 13, 14, 15, 16, 17 +5. Reads each file's `writer_epoch` from schema metadata +6. Entries 13, 14, 15 have `writer_epoch=6` which is <= current epoch (7) -> **valid, will be replayed** +7. Entries 16, 17 have `writer_epoch=7` -> **valid, will be replayed** + +#### Key Points + +1. **No data loss**: Writer A's entries are not discarded. They were written with a valid epoch at the time and will be included in recovery. + +2. **Consistency preserved**: Writer A is prevented from making further writes that could conflict with Writer B. + +3. **Orphaned files are safe**: WAL files from fenced writers remain on storage and are replayed by the new writer. They are only garbage collected after being included in a flushed MemTable that has been merged. + +4. **Epoch validation timing**: Writers check their epoch before manifest updates (MemTable flush), not on every WAL write. This keeps the hot path fast while ensuring consistency at commit boundaries. + +### Appendix 2: Concurrent Merger Example + +This example demonstrates how MemWAL Index and conflict resolution handle concurrent mergers safely. + +#### Initial State + +``` +MemWAL Index: + merged_generations: {region: 5} + +Region manifest (version 1): + current_generation: 8 + flushed_generations: [(6, "abc123_gen_6"), (7, "def456_gen_7")] +``` + +#### Scenario 1: Racing on the Same Generation + +Two mergers both try to merge generation 6 concurrently. + +| Step | Merger A | Merger B | MemWAL Index | +|------|----------|----------|--------------| +| 1 | Reads index: merged_gen=5 | | merged_gen=5 | +| 2 | Reads region manifest | | | +| 3 | Starts merging gen 6 | | | +| 4 | | Reads index: merged_gen=5 | merged_gen=5 | +| 5 | | Reads region manifest | | +| 6 | | Starts merging gen 6 | | +| 7 | Commits (merged_gen=6) | | **merged_gen=6** | +| 8 | | Tries to commit | | +| 9 | | **Conflict**: reads new index | | +| 10 | | Sees merged_gen=6 >= 6, aborts | | +| 11 | | Reloads, continues to gen 7 | | + +Merger B's conflict resolution detected that generation 6 was already merged by checking the MemWAL Index in the conflicting commit. + +#### Scenario 2: Crash After Table Commit + +Merger A crashes after committing to the table. + +| Step | Merger A | Merger B | MemWAL Index | +|------|----------|----------|--------------| +| 1 | Reads index: merged_gen=5 | | merged_gen=5 | +| 2 | Merges gen 6, commits | | **merged_gen=6** | +| 3 | **CRASH** | | merged_gen=6 | +| 4 | | Reads index: merged_gen=6 | merged_gen=6 | +| 5 | | Reads region manifest | | +| 6 | | **Skips gen 6** (already merged) | | +| 7 | | Merges gen 7, commits | **merged_gen=7** | + +The MemWAL Index is the single source of truth. Merger B correctly used it to determine that generation 6 was already merged. + +#### Key Points + +1. **Single source of truth**: `merged_generations` is the authoritative source for merge progress, updated atomically with data. + +2. **Conflict resolution uses MemWAL Index**: When a commit conflicts, the merger checks the conflicting commit's MemWAL Index. + +3. **No progress regression**: Because MemWAL Index is updated atomically with data, concurrent mergers cannot regress the merge progress. + +### Appendix 3: Execution Nodes + +This appendix describes custom execution nodes for MemWAL query execution. + +#### DeduplicateExec + +Deduplicates rows by primary key, keeping the row with highest `(_gen, _rowaddr)`. +Since each dataset has a fixed `_gen` and rows are naturally ordered by `_rowaddr`, this can be implemented as a streaming operator without full materialization. + +#### TakeLastExec + +Efficiently finds the last matching row for a filter predicate without full scan. +If the primary key has a btree index, directly queries the btree to get the result. +Otherwise, scans fragments in reverse order and within each fragment takes the last matching row. +Returns immediately upon finding a match, avoiding unnecessary I/O on earlier fragments. + +#### CoalesceFirstExec + +Returns the first non-empty result from multiple inputs with short-circuit evaluation. +Inputs are evaluated lazily in order; on first non-empty result, remaining inputs are not evaluated. + +#### FilterStaleExec + +Filters out rows that have a newer version in a higher generation. +For each candidate with primary key `pk` from generation G, checks bloom filters of generations > G. +If the bloom filter indicates the key may exist in a newer generation, the candidate is filtered out. +False positives from bloom filters may cause some valid results to be filtered, but this is acceptable for search workloads where approximate results are expected. + +#### BloomFilterGuardExec + +Guards a child execution node with a bloom filter check. +Given a primary key, checks the bloom filter before executing the child node. +If the bloom filter returns negative (key definitely not present), returns empty without executing the child. +If the bloom filter returns positive (key may be present), executes the child node normally. +Used in point lookups to skip unnecessary scans of generations that don't contain the target key. + +### Appendix 4: Index Catchup Example + +This example demonstrates how `index_catchup` enables indexed reads during async index rebuilding. + +#### Scenario Setup + +``` +Generation: 1 2 3 4 5 6 + | | | | | | +State: merged merged merged merged flushed active + | | | | | +Base IVF index: [-- covers 1-3 --] | | + ↑ ↑ ↑ + index_gen=3 merged_gen=4 | + current_gen=6 +``` + +In this example: + +- **Generations 1-4** have been merged to the base table (`merged_gen=4`) +- **Base IVF index** has only been rebuilt to cover generations 1-3 (`index_gen=3`) +- **Generation 4** is in the base table but NOT covered by the base IVF index +- **Generation 5** is flushed to disk (not yet merged to base table) +- **Generation 6** is the active in-memory MemTable + +#### Example Read Strategy for Vector Search + +Without `index_catchup` tracking, the query planner would need to perform an expensive full scan on the base table for generation 4. +With `index_catchup`, the planner knows exactly which data is indexed and can use flushed MemTable indexes for the gap: + +| Data Source | Generations | Strategy | +|-------------|-------------|----------| +| Base table with IVF index | 1-3 | Use base table's IVF index | +| Flushed MemTable gen 4 | 4 | Use flushed MemTable's IVF index | +| Flushed MemTable gen 5 | 5 | Use flushed MemTable's IVF index | +| Active MemTable | 6 | Use in-memory IVF index | + +All data sources provide indexed access, maintaining query performance during async index rebuild. diff --git a/docs/src/images/mem_wal_overview.png b/docs/src/images/mem_wal_overview.png new file mode 100644 index 00000000000..008c84d0724 Binary files /dev/null and b/docs/src/images/mem_wal_overview.png differ diff --git a/docs/src/images/mem_wal_regional.png b/docs/src/images/mem_wal_regional.png new file mode 100644 index 00000000000..451ee95414f Binary files /dev/null and b/docs/src/images/mem_wal_regional.png differ diff --git a/java/lance-jni/src/transaction.rs b/java/lance-jni/src/transaction.rs index c80ef36bffa..59ca0d36eff 100644 --- a/java/lance-jni/src/transaction.rs +++ b/java/lance-jni/src/transaction.rs @@ -559,7 +559,7 @@ fn convert_to_java_operation_inner<'local>( updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge: _, + merged_generations: _, fields_for_preserving_frag_bitmap, update_mode, inserted_rows_filter: _, @@ -1048,7 +1048,7 @@ fn convert_to_rust_operation( updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge: None, + merged_generations: vec![], fields_for_preserving_frag_bitmap, update_mode, inserted_rows_filter: None, diff --git a/protos/table.proto b/protos/table.proto index 02e85f25844..0d73bfdeb94 100644 --- a/protos/table.proto +++ b/protos/table.proto @@ -504,80 +504,176 @@ message FragmentReuseIndexDetails { } } +// ============================================================================ +// MemWAL Index Types +// ============================================================================ + +// Region manifest containing epoch-based fencing and WAL state. +// Each region has exactly one active writer at any time. +message RegionManifest { + // Region identifier (UUID v4). + UUID region_id = 11; + + // Manifest version number. + // Matches the version encoded in the filename. + uint64 version = 1; + + // Region spec ID this region was created with. + // Set at region creation and immutable thereafter. + // A value of 0 indicates a manually-created region not governed by any spec. + uint32 region_spec_id = 10; + + // Writer fencing token - monotonically increasing. + // A writer must increment this when claiming the region. + uint64 writer_epoch = 2; + + // The most recent WAL entry ID that has been flushed to a MemTable. + // During recovery, replay starts from replay_after_wal_id + 1. + uint64 replay_after_wal_id = 3; + + // The most recent WAL entry ID at the time manifest was updated. + // This is a hint, not authoritative - recovery must list files to find actual state. + uint64 wal_id_last_seen = 4; + + // Next generation ID to create (incremented after each MemTable flush). + uint64 current_generation = 6; + + // Field 7 removed: merged_generation moved to MemWalIndexDetails.merged_generations + // which is the authoritative source for merge progress. + + // List of flushed MemTable generations and their directory paths. + repeated FlushedGeneration flushed_generations = 8; +} + +// A flushed MemTable generation and its storage location. +message FlushedGeneration { + // Generation number. + uint64 generation = 1; + + // Directory name relative to the region directory. + string path = 2; +} + +// A region's merged generation, used in MemWalIndexDetails. +message MergedGeneration { + // Region identifier (UUID v4). + UUID region_id = 1; + + // Last generation merged to base table for this region. + uint64 generation = 2; +} + +// Tracks which merged generation a base table index has been rebuilt to cover. +// Used to determine whether to read from flushed MemTable indexes or base table. +message IndexCatchupProgress { + // Name of the base table index (must match an entry in maintained_indexes). + string index_name = 1; + + // Per-region progress: the generation up to which this index covers. + // If a region is not present, the index is assumed to be fully caught up + // (i.e., caught_up_generation >= merged_generation for that region). + repeated MergedGeneration caught_up_generations = 2; +} + +// Index details for MemWAL Index, stored in IndexMetadata.index_details. +// This is the centralized structure for all MemWAL metadata: +// - Configuration (region specs, indexes to maintain) +// - Merge progress (merged generations per region) +// - Region state snapshots +// +// Writers read this index to get configuration before writing. +// Readers read this index to discover regions and their state. +// A background process updates the index periodically to keep region snapshots current. +// +// Region snapshots are stored as a Lance file with one row per region. +// The schema has one column per RegionManifest field, with region fields as columns: +// region_id: fixed_size_binary(16) -- UUID bytes +// version: uint64 +// region_spec_id: uint32 +// writer_epoch: uint64 +// replay_after_wal_id: uint64 +// wal_id_last_seen: uint64 +// current_generation: uint64 +// merged_generation: uint64 +// flushed_generations: list> message MemWalIndexDetails { + // Snapshot timestamp (Unix timestamp in milliseconds). + int64 snapshot_ts_millis = 1; + + // Number of regions in the snapshot. + // Used to determine storage format without reading the snapshot data. + uint32 num_regions = 2; + + // Inline region snapshots for small region counts. + // When num_regions <= threshold (implementation-defined, e.g., 100), + // snapshots are stored inline as serialized bytes. + // Format: Lance file bytes with the region snapshot schema. + optional bytes inline_snapshots = 3; + + // Region specs defining how to derive region identifiers. + // This configuration determines how rows are partitioned into regions. + repeated RegionSpec region_specs = 7; + + // Indexes from the base table to maintain in MemTables. + // These are index names referencing indexes defined on the base table. + // The primary key btree index is always maintained implicitly and + // should not be listed here. + // + // For vector indexes, MemTables inherit quantization parameters (PQ codebook, + // SQ params) from the base table index to ensure distance comparability. + repeated string maintained_indexes = 8; + + // Last generation merged to base table for each region. + // This is updated atomically with merge-insert data commits, enabling + // conflict resolution when multiple mergers operate concurrently. + // + // Note: This is separate from region snapshots because: + // 1. merged_generations is updated by mergers (atomic with data commit) + // 2. region snapshots are updated by background index builder + repeated MergedGeneration merged_generations = 9; + + // Per-index catchup progress tracking. + // When data is merged to the base table, base table indexes are rebuilt + // asynchronously. This field tracks which generation each index covers. + // + // For indexed queries, if an index's caught_up_generation < merged_generation, + // readers should use flushed MemTable indexes for the gap instead of + // scanning unindexed data in the base table. + // + // If an index is not present in this list, it is assumed to be fully caught up. + repeated IndexCatchupProgress index_catchup = 10; +} - repeated MemWal mem_wal_list = 1; +// Region spec definition. +message RegionSpec { + // Unique identifier for this spec within the index. + // IDs are never reused. + uint32 spec_id = 1; - message MemWalId { - // The name of the region that this specific MemWAL is responsible for. - string region = 1; + // Region field definitions that determine how to compute region identifiers. + repeated RegionField fields = 2; +} - // The generation of the MemWAL. - // Every time a new MemWAL is created and an old one is sealed, - // the generation number of the next MemWAL is incremented. - // At any given point of time for all MemWALs of the same name, - // there must be only 1 generation that is not sealed. - uint64 generation = 2; - } +// Region field definition. +message RegionField { + // Unique string identifier for this region field. + string field_id = 1; - // A combination of MemTable and WAL for fast upsert. - message MemWal { - - enum State { - // MemWAL is open and accepting new entries - OPEN = 0; - // When a MemTable is considered full, the writer should update this MemWAL as sealed - // and create a new MemWAL to write to atomically. - SEALED = 1; - // When a MemTable is sealed, it can be flushed asynchronously to disk. - // This state indicates the data has been persisted to disk but not yet merged - // into the source table. - FLUSHED = 2; - // When the flushed data has been merged into the source table. - // After a MemWAL is merged, the cleanup process can delete the WAL. - MERGED = 3; - } - - MemWalId id = 1; - - // The MemTable location, which is likely an in-memory address starting with memory://. - // The actual details of how the MemTable is stored is outside the concern of Lance. - string mem_table_location = 2; - - // the root location of the WAL. - // THe WAL storage durability determines the data durability. - // This location is immutable once set at MemWAL creation time. - string wal_location = 3; - - // All entries in the WAL, serialized as U64Segment. - // Each entry in the WAL has a uint64 sequence ID starting from 0. - // The actual details of how the WAL entry is stored is outside the concern of Lance. - // In most cases this U64Segment should be a simple range. - // Every time the writer starts writing, it must always try to atomically write to the last entry ID + 1. - // If fails due to concurrent writer, it then tries to write to the +2, +3, +4, etc. entry ID until succeed. - // but if there are 2 writers accidentally writing to the same WAL concurrently, - // although one writer will fail to update this index at commit time, - // the WAL entry is already written, - // causing some holes within the U64Segment range. - bytes wal_entries = 4; - - // The current state of the MemWAL, indicating its lifecycle phase. - // States progress: OPEN -> SEALED -> FLUSHED - // OPEN: MemWAL is accepting new WAL entries - // SEALED: MemWAL has been sealed and no longer accepts new WAL entries - // FLUSHED: MemWAL has been flushed to the source Lance table and can be cleaned up - State state = 5; - - // The owner identifier for this MemWAL, used for compare-and-swap operations. - // When a writer wants to perform any operation on this MemWAL, it must provide - // the expected owner_id. This serves as an optimistic lock to prevent concurrent - // writers from interfering with each other. When a new writer starts replay, - // it must first atomically update this owner_id to claim ownership. - // All subsequent operations will fail if the owner_id has changed. - string owner_id = 6; - - // The dataset version that last updated this MemWAL. - // This is set to the new dataset version whenever the MemWAL is created or modified. - uint64 last_updated_dataset_version = 7; - } + // Field IDs referencing source columns in the schema. + repeated int32 source_ids = 2; + + // Well-known region transform name (e.g., "identity", "year", "bucket"). + // Mutually exclusive with expression. + optional string transform = 3; + + // DataFusion SQL expression for custom logic. + // Mutually exclusive with transform. + optional string expression = 4; + + // Output type of the region value (Arrow type name). + string result_type = 5; + + // Transform parameters (e.g., num_buckets for bucket transform). + map parameters = 6; } + diff --git a/protos/transaction.proto b/protos/transaction.proto index 638898f4197..17d96486736 100644 --- a/protos/transaction.proto +++ b/protos/transaction.proto @@ -235,8 +235,8 @@ message Transaction { repeated DataFragment new_fragments = 3; // The ids of the fields that have been modified. repeated uint32 fields_modified = 4; - /// The MemWAL (pre-image) that should be marked as merged after this transaction - MemWalIndexDetails.MemWal mem_wal_to_merge = 5; + /// List of MemWAL region generations to mark as merged after this transaction + repeated MergedGeneration merged_generations = 5; /// The fields that used to judge whether to preserve the new frag's id into /// the frag bitmap of the specified indices. repeated uint32 fields_for_preserving_frag_bitmap = 6; @@ -305,15 +305,12 @@ message Transaction { repeated DataReplacementGroup replacements = 1; } - // Update the state of the MemWal index + // Update the merged generations in MemWAL index. + // This operation is used during merge-insert to atomically record which + // generations have been merged to the base table. message UpdateMemWalState { - - repeated MemWalIndexDetails.MemWal added = 1; - - repeated MemWalIndexDetails.MemWal updated = 2; - - // If a MemWAL is updated, its pre-image should be in the removed list. - repeated MemWalIndexDetails.MemWal removed = 3; + // Regions and generations being marked as merged. + repeated MergedGeneration merged_generations = 1; } // An operation that updates base paths in the dataset. diff --git a/python/src/transaction.rs b/python/src/transaction.rs index b867b428352..4f57bf3dd49 100644 --- a/python/src/transaction.rs +++ b/python/src/transaction.rs @@ -229,7 +229,7 @@ impl FromPyObject<'_> for PyLance { updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge: None, + merged_generations: vec![], fields_for_preserving_frag_bitmap, update_mode, inserted_rows_filter: None, diff --git a/rust/lance-index/src/mem_wal.rs b/rust/lance-index/src/mem_wal.rs index 7ba1cab80c4..71bda425fd8 100644 --- a/rust/lance-index/src/mem_wal.rs +++ b/rust/lance-index/src/mem_wal.rs @@ -1,208 +1,297 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright The Lance Authors -use crate::{Index, IndexType}; +use std::any::Any; +use std::collections::HashMap; +use std::sync::Arc; + use async_trait::async_trait; -use lance_core::cache::DeepSizeOf; +use deepsize::DeepSizeOf; use lance_core::Error; use lance_table::format::pb; -use lance_table::rowids::segment::U64Segment; -use prost::Message; use roaring::RoaringBitmap; use serde::{Deserialize, Serialize}; use snafu::location; -use std::any::Any; -use std::collections::{BTreeMap, HashMap}; -use std::sync::Arc; +use uuid::Uuid; + +use crate::{Index, IndexType}; pub const MEM_WAL_INDEX_NAME: &str = "__lance_mem_wal"; -#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Serialize, Deserialize, DeepSizeOf)] -pub enum State { - Open, - Sealed, - Flushed, - Merged, +/// Type alias for region identifier (UUID v4). +pub type RegionId = Uuid; + +/// A flushed MemTable generation and its storage location. +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] +pub struct FlushedGeneration { + pub generation: u64, + pub path: String, } -impl From for pb::mem_wal_index_details::mem_wal::State { - fn from(state: State) -> Self { - match state { - State::Open => Self::Open, - State::Sealed => Self::Sealed, - State::Flushed => Self::Flushed, - State::Merged => Self::Merged, +impl From<&FlushedGeneration> for pb::FlushedGeneration { + fn from(fg: &FlushedGeneration) -> Self { + Self { + generation: fg.generation, + path: fg.path.clone(), } } } -impl TryFrom for State { - type Error = Error; - - fn try_from(state: pb::mem_wal_index_details::mem_wal::State) -> lance_core::Result { - match state { - pb::mem_wal_index_details::mem_wal::State::Open => Ok(Self::Open), - pb::mem_wal_index_details::mem_wal::State::Sealed => Ok(Self::Sealed), - pb::mem_wal_index_details::mem_wal::State::Flushed => Ok(Self::Flushed), - pb::mem_wal_index_details::mem_wal::State::Merged => Ok(Self::Merged), +impl From for FlushedGeneration { + fn from(fg: pb::FlushedGeneration) -> Self { + Self { + generation: fg.generation, + path: fg.path, } } } -impl TryFrom for State { - type Error = Error; +/// A region's merged generation, used in MemWalIndexDetails. +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash, Serialize, Deserialize)] +pub struct MergedGeneration { + pub region_id: Uuid, + pub generation: u64, +} - fn try_from(value: i32) -> lance_core::Result { - match value { - 0 => Ok(Self::Open), - 1 => Ok(Self::Sealed), - 2 => Ok(Self::Flushed), - 3 => Ok(Self::Merged), - _ => Err(Error::invalid_input( - format!("Unknown MemWAL state value: {}", value), - location!(), - )), - } +impl DeepSizeOf for MergedGeneration { + fn deep_size_of_children(&self, _context: &mut deepsize::Context) -> usize { + 0 // UUID is 16 bytes fixed size, no heap allocations } } -#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Serialize, Deserialize, DeepSizeOf)] -pub struct MemWalId { - pub region: String, - pub generation: u64, +impl MergedGeneration { + pub fn new(region_id: Uuid, generation: u64) -> Self { + Self { + region_id, + generation, + } + } } -impl From<&MemWalId> for pb::mem_wal_index_details::MemWalId { - fn from(mem_wal: &MemWalId) -> Self { +impl From<&MergedGeneration> for pb::MergedGeneration { + fn from(mg: &MergedGeneration) -> Self { Self { - region: mem_wal.region.clone(), - generation: mem_wal.generation, + region_id: Some((&mg.region_id).into()), + generation: mg.generation, } } } -impl TryFrom for MemWalId { +impl TryFrom for MergedGeneration { type Error = Error; - fn try_from(mem_wal: pb::mem_wal_index_details::MemWalId) -> lance_core::Result { + fn try_from(mg: pb::MergedGeneration) -> lance_core::Result { + let region_id = mg.region_id.as_ref().map(Uuid::try_from).ok_or_else(|| { + Error::invalid_input("Missing region_id in MergedGeneration", location!()) + })??; Ok(Self { - region: mem_wal.region.clone(), - generation: mem_wal.generation, + region_id, + generation: mg.generation, }) } } -impl MemWalId { - pub fn new(region: &str, generation: u64) -> Self { +/// Tracks which merged generation a base table index has been rebuilt to cover. +/// Used to determine whether to read from flushed MemTable indexes or base table. +#[derive(Debug, Clone, Default, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] +pub struct IndexCatchupProgress { + pub index_name: String, + pub caught_up_generations: Vec, +} + +impl IndexCatchupProgress { + pub fn new(index_name: String, caught_up_generations: Vec) -> Self { Self { - region: region.to_owned(), - generation, + index_name, + caught_up_generations, } } -} -#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Serialize, Deserialize, DeepSizeOf)] -pub struct MemWal { - pub id: MemWalId, - pub mem_table_location: String, - pub wal_location: String, - pub wal_entries: Vec, - pub state: State, - pub owner_id: String, - pub last_updated_dataset_version: u64, + /// Get the caught up generation for a specific region. + /// Returns None if the region is not present (assumed fully caught up). + pub fn caught_up_generation_for_region(&self, region_id: &Uuid) -> Option { + self.caught_up_generations + .iter() + .find(|mg| &mg.region_id == region_id) + .map(|mg| mg.generation) + } } -impl From<&MemWal> for pb::mem_wal_index_details::MemWal { - fn from(mem_wal: &MemWal) -> Self { +impl From<&IndexCatchupProgress> for pb::IndexCatchupProgress { + fn from(icp: &IndexCatchupProgress) -> Self { Self { - id: Some(pb::mem_wal_index_details::MemWalId::from(&mem_wal.id)), - mem_table_location: mem_wal.mem_table_location.clone(), - wal_location: mem_wal.wal_location.clone(), - wal_entries: mem_wal.wal_entries.clone(), - state: pb::mem_wal_index_details::mem_wal::State::from(mem_wal.state.clone()) as i32, - owner_id: mem_wal.owner_id.clone(), - last_updated_dataset_version: mem_wal.last_updated_dataset_version, + index_name: icp.index_name.clone(), + caught_up_generations: icp + .caught_up_generations + .iter() + .map(|mg| mg.into()) + .collect(), } } } -impl TryFrom for MemWal { +impl TryFrom for IndexCatchupProgress { type Error = Error; - fn try_from(mem_wal: pb::mem_wal_index_details::MemWal) -> lance_core::Result { - let state = State::try_from(mem_wal.state)?; - + fn try_from(icp: pb::IndexCatchupProgress) -> lance_core::Result { Ok(Self { - id: MemWalId::try_from(mem_wal.id.unwrap())?, - mem_table_location: mem_wal.mem_table_location.clone(), - wal_location: mem_wal.wal_location.clone(), - wal_entries: mem_wal.wal_entries, - state, - owner_id: mem_wal.owner_id, - last_updated_dataset_version: mem_wal.last_updated_dataset_version, + index_name: icp.index_name, + caught_up_generations: icp + .caught_up_generations + .into_iter() + .map(MergedGeneration::try_from) + .collect::>()?, }) } } -impl MemWal { - pub fn new_empty( - id: MemWalId, - mem_table_location: &str, - wal_location: &str, - owner_id: &str, - ) -> Self { +/// Region manifest containing epoch-based fencing and WAL state. +/// Each region has exactly one active writer at any time. +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] +pub struct RegionManifest { + pub region_id: Uuid, + pub version: u64, + pub region_spec_id: u32, + pub writer_epoch: u64, + pub replay_after_wal_id: u64, + pub wal_id_last_seen: u64, + pub current_generation: u64, + pub flushed_generations: Vec, +} + +impl DeepSizeOf for RegionManifest { + fn deep_size_of_children(&self, context: &mut deepsize::Context) -> usize { + self.flushed_generations.deep_size_of_children(context) + } +} + +impl From<&RegionManifest> for pb::RegionManifest { + fn from(rm: &RegionManifest) -> Self { Self { - id, - mem_table_location: mem_table_location.to_owned(), - wal_location: wal_location.to_owned(), - wal_entries: pb::U64Segment::from(U64Segment::Range(0..0)).encode_to_vec(), - state: State::Open, - owner_id: owner_id.to_owned(), - last_updated_dataset_version: 0, // placeholder, this will be filled during build_manifest + region_id: Some((&rm.region_id).into()), + version: rm.version, + region_spec_id: rm.region_spec_id, + writer_epoch: rm.writer_epoch, + replay_after_wal_id: rm.replay_after_wal_id, + wal_id_last_seen: rm.wal_id_last_seen, + current_generation: rm.current_generation, + flushed_generations: rm.flushed_generations.iter().map(|fg| fg.into()).collect(), } } +} + +impl TryFrom for RegionManifest { + type Error = Error; - pub fn wal_entries(&self) -> U64Segment { - U64Segment::try_from(pb::U64Segment::decode(self.wal_entries.as_slice()).unwrap()).unwrap() + fn try_from(rm: pb::RegionManifest) -> lance_core::Result { + let region_id = rm.region_id.as_ref().map(Uuid::try_from).ok_or_else(|| { + Error::invalid_input("Missing region_id in RegionManifest", location!()) + })??; + Ok(Self { + region_id, + version: rm.version, + region_spec_id: rm.region_spec_id, + writer_epoch: rm.writer_epoch, + replay_after_wal_id: rm.replay_after_wal_id, + wal_id_last_seen: rm.wal_id_last_seen, + current_generation: rm.current_generation, + flushed_generations: rm + .flushed_generations + .into_iter() + .map(FlushedGeneration::from) + .collect(), + }) } +} + +/// Region field definition. +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] +pub struct RegionField { + pub field_id: String, + pub source_ids: Vec, + pub transform: Option, + pub expression: Option, + pub result_type: String, + pub parameters: HashMap, +} - /// Check if the MemWAL is in the expected state - pub fn check_state(&self, expected: State) -> lance_core::Result<()> { - if self.state != expected { - return Err(Error::invalid_input( - format!( - "MemWAL {:?} is in state {:?}, but expected {:?}", - self.id, self.state, expected - ), - location!(), - )); +impl From<&RegionField> for pb::RegionField { + fn from(rf: &RegionField) -> Self { + Self { + field_id: rf.field_id.clone(), + source_ids: rf.source_ids.clone(), + transform: rf.transform.clone(), + expression: rf.expression.clone(), + result_type: rf.result_type.clone(), + parameters: rf.parameters.clone(), } - Ok(()) } +} - pub fn check_expected_owner_id(&self, expected: &str) -> lance_core::Result<()> { - if self.owner_id != expected { - return Err(Error::invalid_input( - format!( - "MemWAL {:?} has owner_id: {}, but expected {}", - self.id, self.owner_id, expected - ), - location!(), - )); +impl From for RegionField { + fn from(rf: pb::RegionField) -> Self { + Self { + field_id: rf.field_id, + source_ids: rf.source_ids, + transform: rf.transform, + expression: rf.expression, + result_type: rf.result_type, + parameters: rf.parameters, } - Ok(()) } } +/// Region spec definition. #[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] +pub struct RegionSpec { + pub spec_id: u32, + pub fields: Vec, +} + +impl From<&RegionSpec> for pb::RegionSpec { + fn from(rs: &RegionSpec) -> Self { + Self { + spec_id: rs.spec_id, + fields: rs.fields.iter().map(|f| f.into()).collect(), + } + } +} + +impl From for RegionSpec { + fn from(rs: pb::RegionSpec) -> Self { + Self { + spec_id: rs.spec_id, + fields: rs.fields.into_iter().map(RegionField::from).collect(), + } + } +} + +/// Index details for MemWAL Index, stored in IndexMetadata.index_details. +#[derive(Debug, Clone, Default, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] pub struct MemWalIndexDetails { - pub mem_wal_list: Vec, + pub snapshot_ts_millis: i64, + pub num_regions: u32, + pub inline_snapshots: Option>, + pub region_specs: Vec, + pub maintained_indexes: Vec, + pub merged_generations: Vec, + pub index_catchup: Vec, } impl From<&MemWalIndexDetails> for pb::MemWalIndexDetails { fn from(details: &MemWalIndexDetails) -> Self { Self { - mem_wal_list: details.mem_wal_list.iter().map(|m| m.into()).collect(), + snapshot_ts_millis: details.snapshot_ts_millis, + num_regions: details.num_regions, + inline_snapshots: details.inline_snapshots.clone(), + region_specs: details.region_specs.iter().map(|rs| rs.into()).collect(), + maintained_indexes: details.maintained_indexes.clone(), + merged_generations: details + .merged_generations + .iter() + .map(|mg| mg.into()) + .collect(), + index_catchup: details.index_catchup.iter().map(|icp| icp.into()).collect(), } } } @@ -212,42 +301,76 @@ impl TryFrom for MemWalIndexDetails { fn try_from(details: pb::MemWalIndexDetails) -> lance_core::Result { Ok(Self { - mem_wal_list: details - .mem_wal_list + snapshot_ts_millis: details.snapshot_ts_millis, + num_regions: details.num_regions, + inline_snapshots: details.inline_snapshots, + region_specs: details + .region_specs + .into_iter() + .map(RegionSpec::from) + .collect(), + maintained_indexes: details.maintained_indexes, + merged_generations: details + .merged_generations .into_iter() - .map(MemWal::try_from) + .map(MergedGeneration::try_from) + .collect::>()?, + index_catchup: details + .index_catchup + .into_iter() + .map(IndexCatchupProgress::try_from) .collect::>()?, }) } } +/// MemWAL Index provides access to MemWAL configuration and state. #[derive(Debug, Clone, PartialEq, Eq, DeepSizeOf)] pub struct MemWalIndex { - pub mem_wal_map: HashMap>, + pub details: MemWalIndexDetails, } impl MemWalIndex { pub fn new(details: MemWalIndexDetails) -> Self { - let mut mem_wal_map: HashMap> = HashMap::new(); - for mem_wal in details.mem_wal_list.into_iter() { - if let Some(generations) = mem_wal_map.get_mut(&mem_wal.id.region) { - generations.insert(mem_wal.id.generation, mem_wal); - } else { - mem_wal_map.insert( - mem_wal.id.region.clone(), - std::iter::once((mem_wal.id.generation, mem_wal)).collect(), - ); - } - } + Self { details } + } + + pub fn merged_generation_for_region(&self, region_id: &Uuid) -> Option { + self.details + .merged_generations + .iter() + .find(|mg| &mg.region_id == region_id) + .map(|mg| mg.generation) + } + + /// Get the caught up generation for a specific index and region. + /// Returns None if the index is not tracked (assumed fully caught up). + pub fn index_caught_up_generation(&self, index_name: &str, region_id: &Uuid) -> Option { + self.details + .index_catchup + .iter() + .find(|icp| icp.index_name == index_name) + .and_then(|icp| icp.caught_up_generation_for_region(region_id)) + } + + /// Check if an index is fully caught up for a region. + /// Returns true if the index covers all merged data for the region. + pub fn is_index_caught_up(&self, index_name: &str, region_id: &Uuid) -> bool { + let merged_gen = self.merged_generation_for_region(region_id).unwrap_or(0); + let caught_up_gen = self.index_caught_up_generation(index_name, region_id); - Self { mem_wal_map } + // If not tracked in index_catchup, assumed fully caught up + caught_up_gen.is_none_or(|gen| gen >= merged_gen) } } #[derive(Serialize)] struct MemWalStatistics { - num_mem_wal: u64, - num_regions: u64, + num_regions: u32, + num_merged_generations: usize, + num_region_specs: usize, + num_maintained_indexes: usize, + num_index_catchup_entries: usize, } #[async_trait] @@ -262,15 +385,18 @@ impl Index for MemWalIndex { fn as_vector_index(self: Arc) -> lance_core::Result> { Err(Error::NotSupported { - source: "FragReuseIndex is not a vector index".into(), + source: "MemWalIndex is not a vector index".into(), location: location!(), }) } fn statistics(&self) -> lance_core::Result { let stats = MemWalStatistics { - num_mem_wal: self.mem_wal_map.values().map(|m| m.len()).sum::() as u64, - num_regions: self.mem_wal_map.len() as u64, + num_regions: self.details.num_regions, + num_merged_generations: self.details.merged_generations.len(), + num_region_specs: self.details.region_specs.len(), + num_maintained_indexes: self.details.maintained_indexes.len(), + num_index_catchup_entries: self.details.index_catchup.len(), }; serde_json::to_value(stats).map_err(|e| Error::Internal { message: format!("failed to serialize MemWAL index statistics: {}", e), @@ -287,6 +413,6 @@ impl Index for MemWalIndex { } async fn calculate_included_frags(&self) -> lance_core::Result { - unimplemented!() + Ok(RoaringBitmap::new()) } } diff --git a/rust/lance/src/dataset/fragment.rs b/rust/lance/src/dataset/fragment.rs index cfe003a7233..8428cf619b4 100644 --- a/rust/lance/src/dataset/fragment.rs +++ b/rust/lance/src/dataset/fragment.rs @@ -2852,7 +2852,7 @@ mod tests { updated_fragments: vec![updated_fragment1], new_fragments: vec![], fields_modified: fields_modified1, - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: Some(UpdateMode::RewriteColumns), inserted_rows_filter: None, @@ -2925,7 +2925,7 @@ mod tests { updated_fragments: vec![updated_fragment2], new_fragments: vec![], fields_modified: fields_modified2, - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: Some(UpdateMode::RewriteColumns), inserted_rows_filter: None, diff --git a/rust/lance/src/dataset/transaction.rs b/rust/lance/src/dataset/transaction.rs index e5916b7bae1..090ef465727 100644 --- a/rust/lance/src/dataset/transaction.rs +++ b/rust/lance/src/dataset/transaction.rs @@ -48,12 +48,12 @@ use super::write::merge_insert::inserted_rows::KeyExistenceFilter; use super::{blob::BLOB_VERSION_CONFIG_KEY, ManifestWriteConfig}; use crate::dataset::transaction::UpdateMode::RewriteRows; -use crate::index::mem_wal::update_mem_wal_index_in_indices_list; +use crate::index::mem_wal::update_mem_wal_index_merged_generations; use crate::utils::temporal::timestamp_to_nanos; use deepsize::DeepSizeOf; use lance_core::{datatypes::BlobVersion, datatypes::Schema, Error, Result}; use lance_file::{datatypes::Fields, version::LanceFileVersion}; -use lance_index::mem_wal::MemWal; +use lance_index::mem_wal::MergedGeneration; use lance_index::{frag_reuse::FRAG_REUSE_INDEX_NAME, is_system_index}; use lance_io::object_store::ObjectStore; use lance_table::feature_flags::{apply_feature_flags, FLAG_STABLE_ROW_IDS}; @@ -245,8 +245,8 @@ pub enum Operation { new_fragments: Vec, /// The fields that have been modified fields_modified: Vec, - /// The MemWAL (pre-image) that should be marked as merged after this transaction - mem_wal_to_merge: Option, + /// List of MemWAL region generations to mark as merged after this transaction + merged_generations: Vec, /// The fields that used to judge whether to preserve the new frag's id into /// the frag bitmap of the specified indices. fields_for_preserving_frag_bitmap: Vec, @@ -267,11 +267,11 @@ pub enum Operation { schema_metadata_updates: Option, field_metadata_updates: HashMap, }, - /// Update the state of MemWALs. + /// Update merged generations in MemWAL index. + /// This is used during merge-insert to atomically record which + /// generations have been merged to the base table. UpdateMemWalState { - added: Vec, - updated: Vec, - removed: Vec, + merged_generations: Vec, }, /// Clone a dataset. @@ -450,7 +450,7 @@ impl PartialEq for Operation { updated_fragments: a_updated, new_fragments: a_new, fields_modified: a_fields, - mem_wal_to_merge: a_mem_wal_to_merge, + merged_generations: a_merged_generations, fields_for_preserving_frag_bitmap: a_fields_for_preserving_frag_bitmap, update_mode: a_update_mode, inserted_rows_filter: a_inserted_rows_filter, @@ -460,7 +460,7 @@ impl PartialEq for Operation { updated_fragments: b_updated, new_fragments: b_new, fields_modified: b_fields, - mem_wal_to_merge: b_mem_wal_to_merge, + merged_generations: b_merged_generations, fields_for_preserving_frag_bitmap: b_fields_for_preserving_frag_bitmap, update_mode: b_update_mode, inserted_rows_filter: b_inserted_rows_filter, @@ -470,7 +470,7 @@ impl PartialEq for Operation { && compare_vec(a_updated, b_updated) && compare_vec(a_new, b_new) && compare_vec(a_fields, b_fields) - && a_mem_wal_to_merge == b_mem_wal_to_merge + && compare_vec(a_merged_generations, b_merged_generations) && compare_vec( a_fields_for_preserving_frag_bitmap, b_fields_for_preserving_frag_bitmap, @@ -1026,20 +1026,12 @@ impl PartialEq for Operation { } ( Self::UpdateMemWalState { - added: a_added, - updated: a_updated, - removed: a_removed, + merged_generations: a_merged, }, Self::UpdateMemWalState { - added: b_added, - updated: b_updated, - removed: b_removed, + merged_generations: b_merged, }, - ) => { - compare_vec(a_added, b_added) - && compare_vec(a_updated, b_updated) - && compare_vec(a_removed, b_removed) - } + ) => compare_vec(a_merged, b_merged), (Self::Clone { .. }, Self::Append { .. }) => { std::mem::discriminant(self) == std::mem::discriminant(other) } @@ -1711,7 +1703,7 @@ impl Transaction { updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge, + merged_generations, fields_for_preserving_frag_bitmap, update_mode, .. @@ -1923,17 +1915,11 @@ impl Transaction { final_fragments.extend(new_fragments); Self::retain_relevant_indices(&mut final_indices, &schema, &final_fragments); - if let Some(mem_wal_to_merge) = mem_wal_to_merge { - update_mem_wal_index_in_indices_list( - self.read_version, - current_manifest.map_or(1, |m| m.version + 1), + if !merged_generations.is_empty() { + update_mem_wal_index_merged_generations( &mut final_indices, - vec![], - vec![MemWal { - state: lance_index::mem_wal::State::Merged, - ..mem_wal_to_merge.clone() - }], - vec![mem_wal_to_merge.clone()], + current_manifest.map_or(1, |m| m.version + 1), + merged_generations.clone(), )?; } } @@ -2143,18 +2129,11 @@ impl Transaction { final_fragments.extend(unmodified_fragments); } - Operation::UpdateMemWalState { - added, - updated, - removed, - } => { - update_mem_wal_index_in_indices_list( - self.read_version, - current_manifest.map_or(1, |m| m.version + 1), + Operation::UpdateMemWalState { merged_generations } => { + update_mem_wal_index_merged_generations( &mut final_indices, - added.clone(), - updated.clone(), - removed.clone(), + current_manifest.map_or(1, |m| m.version + 1), + merged_generations.clone(), )?; } Operation::UpdateBases { .. } => { @@ -2893,7 +2872,7 @@ impl TryFrom for Transaction { updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge, + merged_generations, fields_for_preserving_frag_bitmap, update_mode, inserted_rows, @@ -2908,7 +2887,10 @@ impl TryFrom for Transaction { .map(Fragment::try_from) .collect::>>()?, fields_modified, - mem_wal_to_merge: mem_wal_to_merge.map(|m| MemWal::try_from(m).unwrap()), + merged_generations: merged_generations + .into_iter() + .map(|m| MergedGeneration::try_from(m).unwrap()) + .collect(), fields_for_preserving_frag_bitmap, update_mode: match update_mode { 0 => Some(UpdateMode::RewriteRows), @@ -3014,23 +2996,11 @@ impl TryFrom for Transaction { .collect::>>()?, }, Some(pb::transaction::Operation::UpdateMemWalState( - pb::transaction::UpdateMemWalState { - added, - updated, - removed, - }, + pb::transaction::UpdateMemWalState { merged_generations }, )) => Operation::UpdateMemWalState { - added: added + merged_generations: merged_generations .into_iter() - .map(|m| MemWal::try_from(m).unwrap()) - .collect(), - updated: updated - .into_iter() - .map(|m| MemWal::try_from(m).unwrap()) - .collect(), - removed: removed - .into_iter() - .map(|m| MemWal::try_from(m).unwrap()) + .map(|m| MergedGeneration::try_from(m).unwrap()) .collect(), }, Some(pb::transaction::Operation::UpdateBases(pb::transaction::UpdateBases { @@ -3225,7 +3195,7 @@ impl From<&Transaction> for pb::Transaction { updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge, + merged_generations, fields_for_preserving_frag_bitmap, update_mode, inserted_rows_filter, @@ -3237,7 +3207,10 @@ impl From<&Transaction> for pb::Transaction { .collect(), new_fragments: new_fragments.iter().map(pb::DataFragment::from).collect(), fields_modified: fields_modified.clone(), - mem_wal_to_merge: mem_wal_to_merge.as_ref().map(|m| m.into()), + merged_generations: merged_generations + .iter() + .map(pb::MergedGeneration::from) + .collect(), fields_for_preserving_frag_bitmap: fields_for_preserving_frag_bitmap.clone(), update_mode: update_mode .as_ref() @@ -3288,23 +3261,11 @@ impl From<&Transaction> for pb::Transaction { .collect(), }) } - Operation::UpdateMemWalState { - added, - updated, - removed, - } => { + Operation::UpdateMemWalState { merged_generations } => { pb::transaction::Operation::UpdateMemWalState(pb::transaction::UpdateMemWalState { - added: added - .iter() - .map(pb::mem_wal_index_details::MemWal::from) - .collect::>(), - updated: updated - .iter() - .map(pb::mem_wal_index_details::MemWal::from) - .collect::>(), - removed: removed + merged_generations: merged_generations .iter() - .map(pb::mem_wal_index_details::MemWal::from) + .map(pb::MergedGeneration::from) .collect::>(), }) } diff --git a/rust/lance/src/dataset/write/commit.rs b/rust/lance/src/dataset/write/commit.rs index 5070ee5e65f..9b5855efb77 100644 --- a/rust/lance/src/dataset/write/commit.rs +++ b/rust/lance/src/dataset/write/commit.rs @@ -758,7 +758,7 @@ mod tests { new_fragments: vec![], removed_fragment_ids: vec![], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, diff --git a/rust/lance/src/dataset/write/merge_insert.rs b/rust/lance/src/dataset/write/merge_insert.rs index 02af35fb519..de1d9c987cc 100644 --- a/rust/lance/src/dataset/write/merge_insert.rs +++ b/rust/lance/src/dataset/write/merge_insert.rs @@ -94,8 +94,7 @@ use lance_datafusion::{ utils::StreamingWriteSource, }; use lance_file::version::LanceFileVersion; -use lance_index::mem_wal::{MemWal, MemWalId}; -use lance_index::metrics::NoOpMetricsCollector; +use lance_index::mem_wal::MergedGeneration; use lance_index::{DatasetIndexExt, IndexCriteria}; use lance_table::format::{Fragment, IndexMetadata, RowIdMeta}; use log::info; @@ -313,9 +312,8 @@ struct MergeInsertParams { delete_not_matched_by_source: WhenNotMatchedBySource, conflict_retries: u32, retry_timeout: Duration, - // If set, this MemWAL should be marked as merged, and will be committed to replace the - // MemWAL that is currently in the index with the same ID. - mem_wal_to_merge: Option, + // List of MemWAL region generations to mark as merged when this commit succeeds. + merged_generations: Vec, // If true, skip auto cleanup during commits. This should be set to true // for high frequency writes to improve performance. This is also useful // if the writer does not have delete permissions and the clean up would @@ -426,7 +424,7 @@ impl MergeInsertBuilder { delete_not_matched_by_source: WhenNotMatchedBySource::Keep, conflict_retries: 10, retry_timeout: Duration::from_secs(30), - mem_wal_to_merge: None, + merged_generations: Vec::new(), skip_auto_cleanup: false, use_index: true, source_dedupe_behavior: SourceDedupeBehavior::Fail, @@ -513,45 +511,11 @@ impl MergeInsertBuilder { self } - /// Indicate that this merge-insert uses data in a flushed MemTable. - /// Once write is completed, the corresponding MemTable should also be marked as merged. - pub async fn mark_mem_wal_as_merged( - &mut self, - mem_wal_id: MemWalId, - expected_owner_id: &str, - ) -> Result<&mut Self> { - if let Some(mem_wal_index) = self - .dataset - .open_mem_wal_index(&NoOpMetricsCollector) - .await? - { - if let Some(generations) = mem_wal_index.mem_wal_map.get(mem_wal_id.region.as_str()) { - if let Some(mem_wal) = generations.get(&mem_wal_id.generation) { - mem_wal.check_state(lance_index::mem_wal::State::Flushed)?; - mem_wal.check_expected_owner_id(expected_owner_id)?; - self.params.mem_wal_to_merge = Some(mem_wal.clone()); - Ok(self) - } else { - Err(Error::invalid_input( - format!( - "Cannot find MemWAL generation {} for region {}", - mem_wal_id.generation, mem_wal_id.region - ), - location!(), - )) - } - } else { - Err(Error::invalid_input( - format!("Cannot find MemWAL for region {}", mem_wal_id.region), - location!(), - )) - } - } else { - Err(Error::NotSupported { - source: "MemWAL is not enabled".into(), - location: location!(), - }) - } + /// Mark MemWAL region generations as merged when this commit succeeds. + /// This updates the merged_generations in the MemWAL Index atomically with the data commit. + pub fn mark_generations_as_merged(&mut self, generations: Vec) -> &mut Self { + self.params.merged_generations.extend(generations); + self } /// Crate a merge insert job @@ -1590,7 +1554,7 @@ impl MergeInsertJob { updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge: self.params.mem_wal_to_merge, + merged_generations: self.params.merged_generations.clone(), fields_for_preserving_frag_bitmap: vec![], // in-place update do not affect preserving frag bitmap update_mode: Some(RewriteColumns), inserted_rows_filter: None, // not implemented for v1 @@ -1661,7 +1625,7 @@ impl MergeInsertJob { // On this path we only make deletions against updated_fragments and will not // modify any field values. fields_modified: vec![], - mem_wal_to_merge: self.params.mem_wal_to_merge, + merged_generations: self.params.merged_generations.clone(), fields_for_preserving_frag_bitmap: full_schema .fields .iter() diff --git a/rust/lance/src/dataset/write/merge_insert/exec/delete.rs b/rust/lance/src/dataset/write/merge_insert/exec/delete.rs index 69fc71246ea..3bc75100b9e 100644 --- a/rust/lance/src/dataset/write/merge_insert/exec/delete.rs +++ b/rust/lance/src/dataset/write/merge_insert/exec/delete.rs @@ -260,7 +260,7 @@ impl ExecutionPlan for DeleteOnlyMergeInsertExec { let merge_stats_holder = self.merge_stats.clone(); let transaction_holder = self.transaction.clone(); let affected_rows_holder = self.affected_rows.clone(); - let mem_wal_to_merge = self.params.mem_wal_to_merge.clone(); + let merged_generations = self.params.merged_generations.clone(); let result_stream = futures::stream::once(async move { let delete_row_addrs = Self::collect_deletions(input_stream, metrics).await?; @@ -275,7 +275,7 @@ impl ExecutionPlan for DeleteOnlyMergeInsertExec { updated_fragments, new_fragments: vec![], fields_modified: vec![], - mem_wal_to_merge, + merged_generations, fields_for_preserving_frag_bitmap: dataset .schema() .fields diff --git a/rust/lance/src/dataset/write/merge_insert/exec/write.rs b/rust/lance/src/dataset/write/merge_insert/exec/write.rs index 3d5527f47bd..87714e6d46d 100644 --- a/rust/lance/src/dataset/write/merge_insert/exec/write.rs +++ b/rust/lance/src/dataset/write/merge_insert/exec/write.rs @@ -856,7 +856,7 @@ impl ExecutionPlan for FullSchemaMergeInsertExec { let transaction_holder = self.transaction.clone(); let affected_rows_holder = self.affected_rows.clone(); let inserted_rows_filter_holder = self.inserted_rows_filter.clone(); - let mem_wal_to_merge = self.params.mem_wal_to_merge.clone(); + let merged_generations = self.params.merged_generations.clone(); let is_primary_key = self.is_primary_key; let updating_row_ids = { let state = merge_state.lock().unwrap(); @@ -927,7 +927,7 @@ impl ExecutionPlan for FullSchemaMergeInsertExec { updated_fragments, new_fragments, fields_modified: vec![], // No fields are modified in schema for upsert - mem_wal_to_merge, + merged_generations, fields_for_preserving_frag_bitmap: dataset .schema() .fields diff --git a/rust/lance/src/dataset/write/update.rs b/rust/lance/src/dataset/write/update.rs index 99c5fc8d6f8..47e86457f7d 100644 --- a/rust/lance/src/dataset/write/update.rs +++ b/rust/lance/src/dataset/write/update.rs @@ -388,7 +388,7 @@ impl UpdateJob { // are moved(deleted and appended). // so we do not need to handle the frag bitmap of the index about it. fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap, update_mode: Some(RewriteRows), inserted_rows_filter: None, diff --git a/rust/lance/src/index/mem_wal.rs b/rust/lance/src/index/mem_wal.rs index bb1d93b3834..6bc596dd42c 100644 --- a/rust/lance/src/index/mem_wal.rs +++ b/rust/lance/src/index/mem_wal.rs @@ -1,21 +1,28 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright The Lance Authors -use crate::dataset::transaction::{Operation, Transaction}; -use crate::index::DatasetIndexInternalExt; -use crate::Dataset; +//! MemWAL Index operations. +//! +//! The MemWAL Index stores: +//! - Configuration (region_specs, maintained_indexes) +//! - Merge progress (merged_generations per region) +//! - Region state snapshots (eventually consistent) +//! +//! Writers no longer update the index on every write. Instead, they update +//! region manifests directly. This module provides functions to: +//! - Load the MemWAL index +//! - Update merged generations (called during merge-insert commits) + +use std::sync::Arc; + use lance_core::{Error, Result}; -use lance_index::mem_wal::{MemWal, MemWalId, MemWalIndex, MemWalIndexDetails, MEM_WAL_INDEX_NAME}; -use lance_index::metrics::NoOpMetricsCollector; -use lance_index::{is_system_index, DatasetIndexExt}; +use lance_index::mem_wal::{MemWalIndex, MemWalIndexDetails, MergedGeneration, MEM_WAL_INDEX_NAME}; use lance_table::format::{pb, IndexMetadata}; -use prost::Message; use snafu::location; -use std::collections::{HashMap, HashSet}; -use std::sync::Arc; use uuid::Uuid; -fn load_mem_wal_index_details(index: IndexMetadata) -> Result { +/// Load MemWalIndexDetails from an IndexMetadata. +pub(crate) fn load_mem_wal_index_details(index: IndexMetadata) -> Result { if let Some(details_any) = index.index_details.as_ref() { if !details_any.type_url.ends_with("MemWalIndexDetails") { return Err(Error::Index { @@ -38,511 +45,66 @@ fn load_mem_wal_index_details(index: IndexMetadata) -> Result Result> { Ok(Arc::new(MemWalIndex::new(load_mem_wal_index_details( index, )?))) } -/// Find the latest generation -pub async fn find_latest_mem_wal_generation( - dataset: &Dataset, - region: &str, -) -> Result> { - let Some(mem_wal_index) = dataset.open_mem_wal_index(&NoOpMetricsCollector).await? else { - return Ok(None); - }; - - let Some(generations) = mem_wal_index.mem_wal_map.get(region) else { - return Ok(None); - }; - - // MemWALs of the same region is ordered increasingly by its generation - if let Some(latest_mem_wal) = generations.values().last() { - Ok(Some(latest_mem_wal.clone())) - } else { - Err(Error::Internal { - message: format!("Encountered MemWAL index mapping that has a region with an empty list of generations: {}", region), - location: location!(), - }) +/// Update merged_generations in the MemWAL index. +/// This is called during merge-insert commits to atomically record which +/// generations have been merged to the base table. +pub(crate) fn update_mem_wal_index_merged_generations( + indices: &mut Vec, + dataset_version: u64, + new_merged_generations: Vec, +) -> Result<()> { + if new_merged_generations.is_empty() { + return Ok(()); } -} - -pub async fn create_mem_wal_generation( - dataset: &mut Dataset, - region: &str, - generation: u64, - new_mem_table_location: &str, - new_wal_location: &str, - owner_id: &str, -) -> Result { - let mem_wal = MemWal::new_empty( - MemWalId::new(region, generation), - new_mem_table_location, - new_wal_location, - owner_id, - ); - let txn = Transaction::new( - dataset.manifest.version, - Operation::UpdateMemWalState { - added: vec![mem_wal.clone()], - updated: vec![], - removed: vec![], - }, - None, - ); - dataset - .apply_commit(txn, &Default::default(), &Default::default()) - .await?; - - Ok(mem_wal) -} - -/// Advance the generation of the MemWAL for the given region. -/// If the MemWAL does not exist, create one with generation 0, and -/// `expected_owner_id` should be None in this case. -/// If the MemWAL exists, seal the one with the latest generation, -/// and open one with the same name and the next generation. -/// If the MemWALIndex structure does not exist, create it along the way. -pub async fn advance_mem_wal_generation( - dataset: &mut Dataset, - region: &str, - new_mem_table_location: &str, - new_wal_location: &str, - expected_owner_id: Option<&str>, - new_owner_id: &str, -) -> Result<()> { - let transaction = if let Some(mem_wal_index) = - dataset.open_mem_wal_index(&NoOpMetricsCollector).await? - { - let (added_mem_wal, updated_mem_wal, removed_mem_wal) = if let Some(generations) = - mem_wal_index.mem_wal_map.get(region) - { - if let Some(latest_mem_wal) = generations.values().last() { - // TODO: technically should check against all WAL locations - if latest_mem_wal.wal_location == new_wal_location { - return Err(Error::invalid_input( - format!( - "Must use a different WAL location from current: {}", - latest_mem_wal.wal_location - ), - location!(), - )); - } + let pos = indices + .iter() + .position(|idx| idx.name == MEM_WAL_INDEX_NAME); - if let Some(expected_owner_id) = expected_owner_id { - latest_mem_wal.check_expected_owner_id(expected_owner_id)?; - } else { - return Err(Error::invalid_input( - format!( - "Expected creating generation 0 for MemWAL region {}, but found current latest MemWAL: {:?}", - region, latest_mem_wal - ), - location!())); - } + let new_meta = if let Some(pos) = pos { + let current_meta = indices.remove(pos); + let mut details = load_mem_wal_index_details(current_meta)?; - if latest_mem_wal.mem_table_location == new_mem_table_location { - return Err(Error::invalid_input( - format!( - "Must use a different MemTable location from current: {}", - latest_mem_wal.mem_table_location - ), - location!(), - )); + // Update merged_generations - for each region, keep the higher generation + for new_mg in new_merged_generations { + if let Some(existing) = details + .merged_generations + .iter_mut() + .find(|mg| mg.region_id == new_mg.region_id) + { + if new_mg.generation > existing.generation { + existing.generation = new_mg.generation; } - - let (updated_mem_wal, removed_mem_wal) = - if latest_mem_wal.state == lance_index::mem_wal::State::Open { - let mut updated_mem_wal = latest_mem_wal.clone(); - updated_mem_wal.state = lance_index::mem_wal::State::Sealed; - (Some(updated_mem_wal), Some(latest_mem_wal.clone())) - } else { - (None, None) - }; - - let added_mem_wal = MemWal::new_empty( - MemWalId::new(region, latest_mem_wal.id.generation + 1), - new_mem_table_location, - new_wal_location, - new_owner_id, - ); - - Ok((added_mem_wal, updated_mem_wal, removed_mem_wal)) } else { - Err(Error::Internal { - message: format!("Encountered MemWAL index mapping that has a region with an empty list of generations: {}", region), - location: location!(), - }) + details.merged_generations.push(new_mg); } - } else { - if let Some(expected_owner_id) = expected_owner_id { - return Err(Error::invalid_input( - format!( - "Expected advancing MemWAL region {} from owner ID {}, but found no generation yet", - region, expected_owner_id - ), - location!())); - } - - Ok(( - MemWal::new_empty( - MemWalId::new(region, 0), - new_mem_table_location, - new_wal_location, - new_owner_id, - ), - None, - None, - )) - }?; - - Transaction::new( - dataset.manifest.version, - Operation::UpdateMemWalState { - added: vec![added_mem_wal], - updated: updated_mem_wal.into_iter().collect(), - removed: removed_mem_wal.into_iter().collect(), - }, - None, - ) - } else { - // this is the first time the MemWAL index is created - if let Some(expected_owner_id) = expected_owner_id { - return Err(Error::invalid_input( - format!( - "Expected advancing MemWAL region {} from owner ID {}, but found no MemWAL index", - region, expected_owner_id - ), - location!())); - } - - Transaction::new( - dataset.manifest.version, - Operation::UpdateMemWalState { - added: vec![MemWal::new_empty( - MemWalId::new(region, 0), - new_mem_table_location, - new_wal_location, - new_owner_id, - )], - updated: vec![], - removed: vec![], - }, - None, - ) - }; - - dataset - .apply_commit(transaction, &Default::default(), &Default::default()) - .await -} - -/// Add a new entry to the MemWAL -pub async fn append_mem_wal_entry( - dataset: &mut Dataset, - mem_wal_region: &str, - mem_wal_generation: u64, - entry_id: u64, - expected_owner_id: &str, -) -> Result { - let mutate = |mem_wal: &MemWal| -> Result { - // Can only append to open MemWALs - mem_wal.check_state(lance_index::mem_wal::State::Open)?; - mem_wal.check_expected_owner_id(expected_owner_id)?; - - let mut updated_mem_wal = mem_wal.clone(); - let wal_entries = updated_mem_wal.wal_entries(); - updated_mem_wal.wal_entries = - pb::U64Segment::from(wal_entries.with_new_high(entry_id)?).encode_to_vec(); - Ok(updated_mem_wal) - }; - - mutate_mem_wal(dataset, mem_wal_region, mem_wal_generation, mutate).await -} - -/// Mark the specific MemWAL as sealed. -/// Typically, it is recommended to call [`advance_mem_wal_generation`] instead. -/// But this will always keep the table in a state with an unsealed MemTable. -/// Calling this function will only seal the current latest MemWAL without opening the next one. -pub async fn mark_mem_wal_as_sealed( - dataset: &mut Dataset, - mem_wal_region: &str, - mem_wal_generation: u64, - expected_owner_id: &str, -) -> Result { - let mutate = |mem_wal: &MemWal| -> Result { - // Can only seal open MemWALs - mem_wal.check_state(lance_index::mem_wal::State::Open)?; - mem_wal.check_expected_owner_id(expected_owner_id)?; - - let mut updated_mem_wal = mem_wal.clone(); - updated_mem_wal.state = lance_index::mem_wal::State::Sealed; - Ok(updated_mem_wal) - }; - - mutate_mem_wal(dataset, mem_wal_region, mem_wal_generation, mutate).await -} - -/// Mark the specific MemWAL as flushed (data on disk but not merged) -pub async fn mark_mem_wal_as_flushed( - dataset: &mut Dataset, - mem_wal_region: &str, - mem_wal_generation: u64, - expected_owner_id: &str, -) -> Result { - let mutate = |mem_wal: &MemWal| -> Result { - // Can only flush sealed MemWALs - mem_wal.check_state(lance_index::mem_wal::State::Sealed)?; - mem_wal.check_expected_owner_id(expected_owner_id)?; - - let mut updated_mem_wal = mem_wal.clone(); - updated_mem_wal.state = lance_index::mem_wal::State::Flushed; - Ok(updated_mem_wal) - }; - - mutate_mem_wal(dataset, mem_wal_region, mem_wal_generation, mutate).await -} - -/// Mark the specific MemWAL as merged (data merged into source table) -pub async fn mark_mem_wal_as_merged( - dataset: &mut Dataset, - mem_wal_region: &str, - mem_wal_generation: u64, - expected_owner_id: &str, -) -> Result { - let mutate = |mem_wal: &MemWal| -> Result { - // Can only merge flushed MemWALs - mem_wal.check_state(lance_index::mem_wal::State::Flushed)?; - mem_wal.check_expected_owner_id(expected_owner_id)?; - - let mut updated_mem_wal = mem_wal.clone(); - updated_mem_wal.state = lance_index::mem_wal::State::Merged; - Ok(updated_mem_wal) - }; - - mutate_mem_wal(dataset, mem_wal_region, mem_wal_generation, mutate).await -} - -/// Mark the specific MemWAL as flushed, in the list of indices in the dataset. -/// This is intended to be used as a part of the Update transaction after resolving all conflicts. -pub(crate) fn update_mem_wal_index_in_indices_list( - dataset_read_version: u64, - dataset_new_version: u64, - indices: &mut Vec, - added: Vec, - updated: Vec, - removed: Vec, -) -> Result<()> { - let new_meta = if let Some(pos) = indices - .iter() - .position(|idx| idx.name == MEM_WAL_INDEX_NAME) - { - let current_meta = indices.remove(pos); - let mut details = load_mem_wal_index_details(current_meta)?; - let removed_set = removed - .iter() - .map(|rm| rm.id.clone()) - .collect::>(); - details - .mem_wal_list - .retain(|m| !removed_set.contains(&m.id)); - - for mut mem_wal in added.into_iter() { - mem_wal.last_updated_dataset_version = dataset_new_version; - details.mem_wal_list.push(mem_wal); - } - - for mut mem_wal in updated.into_iter() { - mem_wal.last_updated_dataset_version = dataset_new_version; - details.mem_wal_list.push(mem_wal); } - new_mem_wal_index_meta(dataset_read_version, details.mem_wal_list)? + new_mem_wal_index_meta(dataset_version, details)? } else { - // This should only happen with new index creation when opening the first MemWAL - if !updated.is_empty() || !removed.is_empty() { - return Err(Error::invalid_input( - "Cannot update MemWAL state without a MemWAL index", - location!(), - )); - } - - let mut added_with_version = Vec::with_capacity(added.len()); - for mut mem_wal in added.into_iter() { - mem_wal.last_updated_dataset_version = dataset_new_version; - added_with_version.push(mem_wal); - } - - new_mem_wal_index_meta(dataset_read_version, added_with_version)? + // Create new MemWAL index with just the merged generations + let details = MemWalIndexDetails { + merged_generations: new_merged_generations, + ..Default::default() + }; + new_mem_wal_index_meta(dataset_version, details)? }; indices.push(new_meta); Ok(()) } -/// Owner ID serves as a pre-check that the MemWAL has not changed owner before commit. -/// Each writer is required to keep an invariant of its owner ID for a MemWAL. -/// At any point in time, there should be only 1 writer that owns the right to mutate the MemWAL, -/// and the owner ID serves as the optimistic lock for it. -/// Specifically, before a writer starts to replay a WAL, it should call this method to claim -/// ownership and stop any additional writes to the MemWAL from other writers. -/// -/// Consider a distributed cluster which currently has node A writing to the table's MemWAL. -/// A network partition happens, node A is not dead but fails the health check. -/// Node B is newly assigned and starts the WAL replay process which modifies the owner ID. -/// In this case, if node A is doing a modification to the same MemWAL including adding an entry, -/// sealing or flushing, advancing the MemWAL generation, it will receive a commit conflict failure. -/// In theory, all the writes from node A should abort after seeing this failure without retrying. -/// However, if the writer decides to retry the operation for any reason (e.g. a bug), without the check, -/// the retry would succeed. The `expected_owner_id` in all write functions serves as the guard to -/// make sure it continues to fail until the write traffic is fully redirected to node B. -pub async fn update_mem_wal_owner( - dataset: &mut Dataset, - region: &str, - generation: u64, - new_owner_id: &str, - new_mem_table_location: Option<&str>, -) -> Result { - let mutate = |mem_wal: &MemWal| -> Result { - if new_owner_id == mem_wal.owner_id { - return Err(Error::invalid_input( - format!( - "Must use a different owner ID from current: {}", - mem_wal.owner_id - ), - location!(), - )); - } - - if let Some(new_mem_table_location) = new_mem_table_location { - if new_mem_table_location == mem_wal.mem_table_location { - return Err(Error::invalid_input( - format!( - "Must use a different MemTable location from current: {}", - mem_wal.mem_table_location - ), - location!(), - )); - } - } - - let mut updated_mem_wal = mem_wal.clone(); - updated_mem_wal.owner_id = new_owner_id.to_owned(); - if let Some(new_mem_table_location) = new_mem_table_location { - updated_mem_wal.mem_table_location = new_mem_table_location.to_owned(); - } - Ok(updated_mem_wal) - }; - - mutate_mem_wal(dataset, region, generation, mutate).await -} - -/// Trim all the MemWALs that are already merged. -pub async fn trim_mem_wal_index(dataset: &mut Dataset) -> Result<()> { - if let Some(mem_wal_index) = dataset.open_mem_wal_index(&NoOpMetricsCollector).await? { - let indices = dataset.load_indices().await?; - - // group by name to get the latest version of each index - // For delta indices, we take the highest dataset version - let mut index_versions = HashMap::new(); - for index in indices.iter() { - if !is_system_index(index) { - let current_version = index_versions.entry(index.name.clone()).or_insert(0); - *current_version = (*current_version).max(index.dataset_version); - } - } - - let min_index_dataset_version = index_versions.values().min().copied().unwrap_or(u64::MAX); - - let mut removed = Vec::new(); - for (_, generations) in mem_wal_index.mem_wal_map.iter() { - for (_, mem_wal) in generations.iter() { - if mem_wal.state == lance_index::mem_wal::State::Merged { - // all indices are caught up, can trim it - if mem_wal.last_updated_dataset_version <= min_index_dataset_version { - removed.push(mem_wal.clone()); - } - } - } - } - - let transaction = Transaction::new( - dataset.manifest.version, - Operation::UpdateMemWalState { - added: vec![], - updated: vec![], - removed, - }, - None, - ); - - dataset - .apply_commit(transaction, &Default::default(), &Default::default()) - .await - } else { - Err(Error::NotSupported { - source: "MemWAL is not enabled".into(), - location: location!(), - }) - } -} - -async fn mutate_mem_wal( - dataset: &mut Dataset, - region: &str, - generation: u64, - mutate: F, -) -> Result -where - F: Fn(&MemWal) -> Result, -{ - if let Some(mem_wal_index) = dataset.open_mem_wal_index(&NoOpMetricsCollector).await? { - if let Some(generations) = mem_wal_index.mem_wal_map.get(region) { - if let Some(mem_wal) = generations.get(&generation) { - let updated_mem_wal = mutate(mem_wal)?; - - let transaction = Transaction::new( - dataset.manifest.version, - Operation::UpdateMemWalState { - added: vec![], - updated: vec![updated_mem_wal.clone()], - removed: vec![mem_wal.clone()], - }, - None, - ); - - dataset - .apply_commit(transaction, &Default::default(), &Default::default()) - .await?; - - Ok(updated_mem_wal) - } else { - Err(Error::invalid_input( - format!( - "Cannot find MemWAL generation {} for region {}", - generation, region - ), - location!(), - )) - } - } else { - Err(Error::invalid_input( - format!("Cannot find MemWAL for region {}", region), - location!(), - )) - } - } else { - Err(Error::NotSupported { - source: "MemWAL is not enabled".into(), - location: location!(), - }) - } -} - +/// Create a new MemWAL index metadata entry. pub(crate) fn new_mem_wal_index_meta( dataset_version: u64, - new_mem_wal_list: Vec, + details: MemWalIndexDetails, ) -> Result { Ok(IndexMetadata { uuid: Uuid::new_v4(), @@ -551,9 +113,7 @@ pub(crate) fn new_mem_wal_index_meta( dataset_version, fragment_bitmap: None, index_details: Some(Arc::new(prost_types::Any::from_msg( - &pb::MemWalIndexDetails::from(&MemWalIndexDetails { - mem_wal_list: new_mem_wal_list, - }), + &pb::MemWalIndexDetails::from(&details), )?)), index_version: 0, created_at: Some(chrono::Utc::now()), @@ -564,2016 +124,375 @@ pub(crate) fn new_mem_wal_index_meta( #[cfg(test)] mod tests { use super::*; - use crate::dataset::{WriteDestination, WriteMode, WriteParams}; - use crate::index::vector::VectorIndexParams; - use crate::utils::test::{DatagenExt, FragmentCount, FragmentRowCount}; - use arrow_array::types::{Float32Type, Int32Type}; - use lance_datafusion::datagen::DatafusionDatagenExt; - use lance_datagen::{BatchCount, Dimension, RowCount}; - use lance_index::mem_wal::{MemWalId, MEM_WAL_INDEX_NAME}; - use lance_index::optimize::OptimizeOptions; - use lance_index::{DatasetIndexExt, Index}; - use lance_linalg::distance::MetricType; - - #[tokio::test] - async fn test_advance_mem_wal_generation() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Initially, there should be no MemWAL index - let indices = dataset.load_indices().await.unwrap(); - assert!(!indices.iter().any(|idx| idx.name == MEM_WAL_INDEX_NAME)); - - // First call to advance_mem_wal_generation should create the MemWAL index and generation 0 - let initial_version = dataset.manifest.version; - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - // Verify the MemWAL index was created - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should be created"); + use std::sync::Arc; - // Load and verify the MemWAL index details - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!(mem_wal_details.mem_wal_list.len(), 1); - let mem_wal_index = open_mem_wal_index(mem_wal_index_meta.clone()).unwrap(); - let stats = mem_wal_index.statistics().unwrap(); - assert_eq!( - serde_json::to_string(&stats).unwrap(), - dataset.index_statistics(MEM_WAL_INDEX_NAME).await.unwrap() - ); + use arrow_array::{Int32Array, RecordBatch}; + use arrow_schema::{DataType, Field, Schema}; + use lance_index::DatasetIndexExt; - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!(mem_wal.id.region, "GLOBAL"); - assert_eq!(mem_wal.id.generation, 0); - assert_eq!(mem_wal.mem_table_location, "mem_table_location_0"); - assert_eq!(mem_wal.wal_location, "wal_location_0"); - assert_eq!(mem_wal.state, lance_index::mem_wal::State::Open); - assert_eq!(mem_wal.last_updated_dataset_version, initial_version + 1); + use crate::dataset::transaction::{Operation, Transaction}; + use crate::dataset::{CommitBuilder, InsertBuilder, WriteParams}; - // Second call to advance_mem_wal_generation should seal generation 0 and create generation 1 - let version_before_second_advance = dataset.manifest.version; - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", + async fn test_dataset() -> crate::Dataset { + let write_params = WriteParams { + max_rows_per_file: 10, + ..Default::default() + }; + let data = RecordBatch::try_new( + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, true), + ])), + vec![ + Arc::new(Int32Array::from_iter_values(0..10_i32)), + Arc::new(Int32Array::from_iter_values(std::iter::repeat_n(0, 10))), + ], ) - .await .unwrap(); + InsertBuilder::new("memory://test_mem_wal") + .with_params(&write_params) + .execute(vec![data]) + .await + .unwrap() + } - // Verify the MemWAL index now has two generations - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); + /// Test that UpdateMemWalState with lower generation than committed fails without retry. + /// Per spec: If committed_generation >= to_commit_generation, abort without retry. + #[tokio::test] + async fn test_update_mem_wal_state_conflict_lower_generation_no_retry() { + let dataset = test_dataset().await; + let region = Uuid::new_v4(); - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!(mem_wal_details.mem_wal_list.len(), 2); + // First commit UpdateMemWalState with generation 10 + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) + .await + .unwrap(); - // Find generation 0 (should be sealed) and generation 1 (should be unsealed) - let gen_0 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 0) - .expect("Generation 0 should exist"); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); + // Try to commit UpdateMemWalState with generation 5 (lower than 10) + // This should fail with non-retryable conflict + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, + None, + ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; - // Verify generation 0 is sealed - assert_eq!(gen_0.id.region, "GLOBAL"); - assert_eq!(gen_0.id.generation, 0); - assert_eq!(gen_0.mem_table_location, "mem_table_location_0"); - assert_eq!(gen_0.wal_location, "wal_location_0"); - assert_eq!(gen_0.state, lance_index::mem_wal::State::Sealed); - // Verify the sealed MemWAL has updated version - assert_eq!( - gen_0.last_updated_dataset_version, - version_before_second_advance + 1 + assert!( + matches!(result, Err(crate::Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict for lower generation, got {:?}", + result ); + } - // Verify generation 1 is unsealed - assert_eq!(gen_1.id.region, "GLOBAL"); - assert_eq!(gen_1.id.generation, 1); - assert_eq!(gen_1.mem_table_location, "mem_table_location_1"); - assert_eq!(gen_1.wal_location, "wal_location_1"); - assert_eq!(gen_1.state, lance_index::mem_wal::State::Open); - // Verify the new MemWAL has correct version - assert_eq!( - gen_1.last_updated_dataset_version, - version_before_second_advance + 1 + /// Test that UpdateMemWalState with equal generation as committed fails without retry. + #[tokio::test] + async fn test_update_mem_wal_state_conflict_equal_generation_no_retry() { + let dataset = test_dataset().await; + let region = Uuid::new_v4(); + + // First commit UpdateMemWalState with generation 10 + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, ); + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) + .await + .unwrap(); - // Test that using the same MemTable location should fail - let result = advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", // Same as current generation - "wal_location_2", // Different WAL location - Some("owner_1"), - "owner_2", - ) - .await; - assert!( - result.is_err(), - "Should fail when using same MemTable location as current generation" + // Try to commit UpdateMemWalState with generation 10 (equal) + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; - // Test that using the same WAL location should fail - let result = advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_2", // Different MemTable location - "wal_location_1", // Same as current generation - Some("owner_1"), - "owner_2", - ) - .await; assert!( - result.is_err(), - "Should fail when using same WAL location as current generation" + matches!(result, Err(crate::Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict for equal generation, got {:?}", + result ); } + /// Test that UpdateMemWalState with higher generation than committed is retryable. + /// Per spec: If committed_generation < to_commit_generation, retry is allowed. #[tokio::test] - async fn test_append_new_entry_to_mem_wal() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) + async fn test_update_mem_wal_state_conflict_higher_generation_retryable() { + let dataset = test_dataset().await; + let region = Uuid::new_v4(); + + // First commit UpdateMemWalState with generation 5 + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, + None, + ); + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) .await .unwrap(); - // Test failure case: MemWAL is not enabled - let result = append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0").await; - assert!(result.is_err(), "Should fail when MemWAL is not enabled"); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", + // Try to commit UpdateMemWalState with generation 10 (higher than 5) + // This should fail with retryable conflict + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, None, - "owner_0", - ) - .await - .unwrap(); + ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; - // Test failure case: region doesn't exist - let result = append_mem_wal_entry(&mut dataset, "NONEXISTENT", 0, 123, "owner_0").await; - assert!(result.is_err(), "Should fail when region doesn't exist"); + assert!( + matches!(result, Err(crate::Error::RetryableCommitConflict { .. })), + "Expected retryable conflict for higher generation, got {:?}", + result + ); + } - // Test failure case: generation doesn't exist - let result = append_mem_wal_entry(&mut dataset, "GLOBAL", 999, 123, "owner_0").await; - assert!(result.is_err(), "Should fail when generation doesn't exist"); + /// Test that UpdateMemWalState on different regions don't conflict. + #[tokio::test] + async fn test_update_mem_wal_state_different_regions_no_conflict() { + let dataset = test_dataset().await; + let region1 = Uuid::new_v4(); + let region2 = Uuid::new_v4(); - // Test success case: append entry to generation 0 - let version_before_append = dataset.manifest.version; - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") + // First commit UpdateMemWalState for region1 + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region1, 10)], + }, + None, + ); + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) .await .unwrap(); - // Verify the entry was added - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; + // Commit UpdateMemWalState for region2 based on old version + // This should succeed because different regions don't conflict + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region2, 5)], + }, + None, + ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; - // Check that the WAL entries contain the entry_id - let wal_entries = mem_wal.wal_entries(); assert!( - wal_entries.contains(123), - "WAL entries should contain entry_id 123" - ); - // Verify the MemWAL version was updated after append - assert_eq!( - mem_wal.last_updated_dataset_version, - version_before_append + 1 + result.is_ok(), + "Expected success for different regions, got {:?}", + result ); - // Test appending multiple entries - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); - let version_after_second_append = dataset.manifest.version; - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 789, "owner_0") + // Verify both regions are in the index + let dataset = result.unwrap(); + let mem_wal_idx = dataset + .load_indices() .await - .unwrap(); - - // Verify all entries were added - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices + .unwrap() .iter() .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); + .unwrap() + .clone(); + let details = load_mem_wal_index_details(mem_wal_idx).unwrap(); + assert_eq!(details.merged_generations.len(), 2); + } - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; + /// Test that CreateIndex of MemWalIndex can be rebased against UpdateMemWalState. + /// The merged_generations from UpdateMemWalState should be merged into CreateIndex. + #[tokio::test] + async fn test_create_index_rebase_against_update_mem_wal_state() { + let dataset = test_dataset().await; + let region = Uuid::new_v4(); - let wal_entries = mem_wal.wal_entries(); - assert!( - wal_entries.contains(123), - "WAL entries should contain entry_id 123" - ); - assert!( - wal_entries.contains(456), - "WAL entries should contain entry_id 456" - ); - assert!( - wal_entries.contains(789), - "WAL entries should contain entry_id 789" - ); - // Verify the MemWAL version was updated after the last append - assert_eq!( - mem_wal.last_updated_dataset_version, - version_after_second_append + 1 + // First commit UpdateMemWalState with generation 10 + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, ); - - // Test failure case: cannot append to sealed MemWAL - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) .await .unwrap(); - let result = append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 999, "owner_0").await; - assert!( - result.is_err(), - "Should fail when trying to append to sealed MemWAL" - ); - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Sealed, but expected Open"), - "Error message should indicate the MemWAL is sealed, got: {}", error); + // CreateIndex of MemWalIndex based on old version (before UpdateMemWalState) + // This should succeed and merge the generations + let details = MemWalIndexDetails { + num_regions: 1, + ..Default::default() + }; + let mem_wal_index = new_mem_wal_index_meta(dataset.manifest.version - 1, details).unwrap(); + + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::CreateIndex { + new_indices: vec![mem_wal_index], + removed_indices: vec![], + }, + None, + ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; - // Test failure case: cannot append to flushed MemWAL - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - let result = append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 999, "owner_0").await; assert!( - result.is_err(), - "Should fail when trying to append to flushed MemWAL" + result.is_ok(), + "Expected CreateIndex to succeed with rebase, got {:?}", + result ); - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Flushed, but expected Open"), - "Error message should indicate the MemWAL is flushed, got: {}", error); + // Verify the merged_generations from UpdateMemWalState were merged into CreateIndex + let dataset = result.unwrap(); + let mem_wal_idx = dataset + .load_indices() + .await + .unwrap() + .iter() + .find(|idx| idx.name == MEM_WAL_INDEX_NAME) + .unwrap() + .clone(); + let details = load_mem_wal_index_details(mem_wal_idx).unwrap(); + assert_eq!(details.merged_generations.len(), 1); + assert_eq!(details.merged_generations[0].region_id, region); + assert_eq!(details.merged_generations[0].generation, 10); + assert_eq!(details.num_regions, 1); // Config from CreateIndex preserved } + /// Test that UpdateMemWalState against CreateIndex of MemWalIndex checks generations. #[tokio::test] - async fn test_seal_mem_wal() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) + async fn test_update_mem_wal_state_against_create_index_lower_generation() { + let dataset = test_dataset().await; + let region = Uuid::new_v4(); + + // First commit CreateIndex of MemWalIndex with merged_generations + let details = MemWalIndexDetails { + merged_generations: vec![MergedGeneration::new(region, 10)], + ..Default::default() + }; + let mem_wal_index = new_mem_wal_index_meta(dataset.manifest.version, details).unwrap(); + + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::CreateIndex { + new_indices: vec![mem_wal_index], + removed_indices: vec![], + }, + None, + ); + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) .await .unwrap(); - // Test failure case: MemWAL is not enabled - let result = mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!(result.is_err(), "Should fail when MemWAL is not enabled"); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", + // Try UpdateMemWalState with lower generation + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, None, - "owner_0", - ) - .await - .unwrap(); + ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; - // Test failure case: region doesn't exist - let result = mark_mem_wal_as_sealed(&mut dataset, "NONEXISTENT", 0, "owner_0").await; - assert!(result.is_err(), "Should fail when region doesn't exist"); + assert!( + matches!(result, Err(crate::Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict when UpdateMemWalState generation is lower than CreateIndex, got {:?}", + result + ); + } - // Test failure case: generation doesn't exist - let result = mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 999, "owner_0").await; - assert!(result.is_err(), "Should fail when generation doesn't exist"); + #[test] + fn test_update_merged_generations() { + let mut indices = Vec::new(); + let region1 = Uuid::new_v4(); + let region2 = Uuid::new_v4(); - // Verify generation 0 is initially unsealed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Open, - "Generation 0 should initially be open" - ); - - // Test success case: seal generation 0 - let version_before_seal = dataset.manifest.version; - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Verify generation 0 is now sealed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Sealed, - "Generation 0 should now be sealed" - ); - // Verify the MemWAL version was updated after sealing - assert_eq!( - mem_wal.last_updated_dataset_version, - version_before_seal + 1 - ); - - // Create a new generation and test sealing it - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await - .unwrap(); - - // Verify generation 1 is unsealed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Open, - "Generation 1 should be open" - ); - - // Seal generation 1 - let version_before_seal_gen1 = dataset.manifest.version; - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - - // Verify it's sealed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Sealed, - "Generation 1 should be sealed" - ); - // Verify the MemWAL version was updated after sealing generation 1 - assert_eq!( - gen_1.last_updated_dataset_version, - version_before_seal_gen1 + 1 - ); - - // Test that sealing an already sealed MemWAL should fail - let result = mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 1, "owner_1").await; - assert!( - result.is_err(), - "Should fail when trying to seal an already sealed MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 1 } is in state Sealed, but expected Open"), - "Error message should indicate the MemWAL is not open, got: {}", error); - - // Test that sealing an already flushed MemWAL should fail - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - let result = mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when trying to seal an already flushed MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Flushed, but expected Open"), - "Error message should indicate the MemWAL is already flushed, got: {}", error); - } - - #[tokio::test] - async fn test_flush_and_merge_mem_wal() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Test failure case: MemWAL is not enabled - let result = mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!(result.is_err(), "Should fail when MemWAL is not enabled"); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Test failure case: region doesn't exist - let result = mark_mem_wal_as_flushed(&mut dataset, "NONEXISTENT", 0, "owner_0").await; - assert!(result.is_err(), "Should fail when region doesn't exist"); - - // Test failure case: generation doesn't exist - let result = mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 999, "owner_0").await; - assert!(result.is_err(), "Should fail when generation doesn't exist"); - - // Verify generation 0 is initially unflushed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Open, - "Generation 0 should initially be open" - ); - - // Test failure case: cannot flush unsealed MemWAL - let result = mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when trying to flush unsealed MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Open, but expected Sealed"), - "Error message should indicate the MemWAL is not sealed, got: {}", error); - - // Seal generation 0 first - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Test success case: mark sealed generation 0 as flushed - let version_before_flush = dataset.manifest.version; - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Verify generation 0 is now flushed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Flushed, - "Generation 0 should now be flushed" - ); - // Verify the MemWAL version was updated after flushing - assert_eq!( - mem_wal.last_updated_dataset_version, - version_before_flush + 1 - ); - - // Test failure case: cannot flush already flushed MemWAL - let result = mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when trying to flush already flushed MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Flushed, but expected Sealed"), - "Error message should indicate the MemWAL is already flushed, got: {}", error); - - // Test success case: mark flushed generation 0 as merged - let version_before_merge = dataset.manifest.version; - mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Verify generation 0 is now merged - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Merged, - "Generation 0 should now be merged" - ); - // Verify the MemWAL version was updated after merging - assert_eq!( - mem_wal.last_updated_dataset_version, - version_before_merge + 1 - ); - - // Test failure case: cannot merge already merged MemWAL - let result = mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when trying to merge already merged MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Merged, but expected Flushed"), - "Error message should indicate the MemWAL is already merged, got: {}", error); - } - - #[tokio::test] - async fn test_update_mem_wal_owner() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Test failure case: MemWAL is not enabled - let result = update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 0, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await; - assert!(result.is_err(), "Should fail when MemWAL is not enabled"); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Test failure case: region doesn't exist - let result = update_mem_wal_owner( - &mut dataset, - "NONEXISTENT", - 0, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await; - assert!(result.is_err(), "Should fail when region doesn't exist"); - - // Test failure case: generation doesn't exist - let result = update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 999, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await; - assert!(result.is_err(), "Should fail when generation doesn't exist"); - - // Test failure case: cannot replay with same MemTable location - let result = update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 0, - "new_owner_id", - Some("mem_table_location_0"), - ) - .await; - assert!( - result.is_err(), - "Should fail when using same MemTable location" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!( - error.to_string().contains( - "Must use a different MemTable location from current: mem_table_location_0" - ), - "Error message should indicate the MemTable location must be different, got: {}", - error - ); - - // Test success case: start replay with different MemTable location - let version_before_owner_update = dataset.manifest.version; - update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 0, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await - .unwrap(); - - // Verify the MemTable location was updated - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.mem_table_location, "new_mem_table_location", - "MemTable location should be updated" - ); - // Verify the MemWAL version was updated after owner change - assert_eq!( - mem_wal.last_updated_dataset_version, - version_before_owner_update + 1 - ); - - // Test success case: can replay generation 1 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "new_mem_table_location_1", - "wal_location_1", - Some("new_owner_id"), - "owner_1", - ) - .await - .unwrap(); - - let version_before_gen1_owner_update = dataset.manifest.version; - update_mem_wal_owner( - &mut dataset, - "GLOBAL", + // First update - creates new index + update_mem_wal_index_merged_generations( + &mut indices, 1, - "owner_1_new", - Some("mem_table_location_1"), + vec![MergedGeneration::new(region1, 5)], ) - .await .unwrap(); - // Verify the MemTable location was updated for generation 1 - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - - assert_eq!( - gen_1.mem_table_location, "mem_table_location_1", - "Generation 1 MemTable location should be updated" - ); - // Verify the MemWAL version was updated after generation 1 owner change - assert_eq!( - gen_1.last_updated_dataset_version, - version_before_gen1_owner_update + 1 - ); - } - - #[tokio::test] - async fn test_trim_mem_wal_index_with_reindex() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Test failure case: MemWAL is not enabled - let result = trim_mem_wal_index(&mut dataset).await; - assert!(result.is_err(), "Should fail when MemWAL is not enabled"); - - // Create MemWAL index and multiple generations - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await - .unwrap(); - - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_2", - "wal_location_2", - Some("owner_1"), - "owner_2", - ) - .await - .unwrap(); - - // Verify we have 3 generations initially - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!( - mem_wal_details.mem_wal_list.len(), - 3, - "Should have 3 generations initially" - ); - - // flush and merge generation 0 - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Test case 1: No indices exist (besides MemWAL index itself) - // Should trim merged MemWAL since no other indices exist - trim_mem_wal_index(&mut dataset).await.unwrap(); - - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); + assert_eq!(indices.len(), 1); + let details = load_mem_wal_index_details(indices[0].clone()).unwrap(); + assert_eq!(details.merged_generations.len(), 1); + assert_eq!(details.merged_generations[0].region_id, region1); + assert_eq!(details.merged_generations[0].generation, 5); - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!( - mem_wal_details.mem_wal_list.len(), + // Second update - updates existing region + update_mem_wal_index_merged_generations( + &mut indices, 2, - "Should have 2 generations after trimming (no other indices)" - ); - - // Verify generation 0 was removed - let gen_0_exists = mem_wal_details - .mem_wal_list - .iter() - .any(|m| m.id.generation == 0); - assert!(!gen_0_exists, "Generation 0 should be removed"); - - // Test case 2: Create index after MemWAL flush, then flush another generation - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_3", - "wal_location_3", - Some("owner_2"), - "owner_3", + vec![MergedGeneration::new(region1, 10)], ) - .await .unwrap(); - // Seal, flush and merge generation 1 - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - - // Create an index after the MemWAL was merged - dataset - .create_index( - &["i"], - lance_index::IndexType::Scalar, - Some("scalar_after".into()), - &lance_index::scalar::ScalarIndexParams::default(), - false, - ) - .await - .unwrap(); - - // Should trim the merged MemWAL since the index was created after it - trim_mem_wal_index(&mut dataset).await.unwrap(); - - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!( - mem_wal_details.mem_wal_list.len(), - 2, - "Should have 2 generations after trimming (index created after MemWAL)" - ); - - // Verify generation 1 was removed - let gen_1_exists = mem_wal_details - .mem_wal_list - .iter() - .any(|m| m.id.generation == 1); - assert!(!gen_1_exists, "Generation 1 should be removed"); - - // Test case 3: Create index before MemWAL flush - // Create another index before flushing the next generation - dataset - .create_index( - &["i"], - lance_index::IndexType::Scalar, - Some("scalar_before".into()), - &lance_index::scalar::ScalarIndexParams::default(), - false, - ) - .await - .unwrap(); - - // Now flush and merge generation 2 (created before the vector index) - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 2, "owner_2") - .await - .unwrap(); - mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 2, "owner_2") - .await - .unwrap(); - - // Should NOT trim generation 2 since the index was created before it - trim_mem_wal_index(&mut dataset).await.unwrap(); - - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!( - mem_wal_details.mem_wal_list.len(), - 2, - "Should still have 2 generations (index created before MemWAL, so cannot trim)" - ); - - // Verify generation 2 still exists - let gen_2_exists = mem_wal_details - .mem_wal_list - .iter() - .any(|m| m.id.generation == 2); - assert!(gen_2_exists, "Generation 2 should still exist"); - } - - #[tokio::test] - async fn test_trim_mem_wal_index_with_delta_index() { - // Create a dataset with enough data for vector index clustering - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(5), FragmentRowCount::from(100)) - .await - .unwrap(); - - // Create initial vector index - dataset - .create_index( - &["vec"], - lance_index::IndexType::Vector, - Some("vector_index".into()), - &VectorIndexParams::ivf_pq(8, 8, 8, MetricType::Cosine, 50), - false, - ) - .await - .unwrap(); + assert_eq!(indices.len(), 1); + let details = load_mem_wal_index_details(indices[0].clone()).unwrap(); + assert_eq!(details.merged_generations.len(), 1); + assert_eq!(details.merged_generations[0].generation, 10); - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", + // Third update - adds new region + update_mem_wal_index_merged_generations( + &mut indices, + 3, + vec![MergedGeneration::new(region2, 3)], ) - .await .unwrap(); - // Seal the MemWAL - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Append new data files to the dataset (without rewriting existing files) - let new_data = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col( - "i", - lance_datagen::array::step_custom::(500, 1000), - ) - .into_reader_rows(RowCount::from(100), BatchCount::from(5)); + assert_eq!(indices.len(), 1); + let details = load_mem_wal_index_details(indices[0].clone()).unwrap(); + assert_eq!(details.merged_generations.len(), 2); - // Append some new data - let write_params = WriteParams { - mode: WriteMode::Append, - ..WriteParams::default() - }; - dataset = Dataset::write( - new_data, - WriteDestination::Dataset(Arc::new(dataset)), - Some(write_params), + // Fourth update - lower generation should not update + update_mem_wal_index_merged_generations( + &mut indices, + 4, + vec![MergedGeneration::new(region1, 8)], // lower than 10 ) - .await .unwrap(); - // Flush and merge the MemWAL separately - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Verify the MemWAL is now merged - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices + let details = load_mem_wal_index_details(indices[0].clone()).unwrap(); + let r1_mg = details + .merged_generations .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!(mem_wal_details.mem_wal_list.len(), 1); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!(mem_wal.state, lance_index::mem_wal::State::Merged); - - // Now use optimize_indices to create delta index (this is how delta indices are actually created) - dataset - .optimize_indices(&OptimizeOptions::append()) - .await + .find(|mg| mg.region_id == region1) .unwrap(); - - // Verify we now have multiple indices with the same name (delta indices) - let indices = dataset.load_indices().await.unwrap(); - let vector_indices: Vec<_> = indices - .iter() - .filter(|idx| idx.name == "vector_index") - .collect(); - assert_eq!(vector_indices.len(), 2); - // If we have delta indices, verify they work correctly - // Verify the delta index has a higher dataset version than the original - let mut versions: Vec<_> = vector_indices - .iter() - .map(|idx| idx.dataset_version) - .collect(); - versions.sort(); - assert!( - versions[versions.len() - 1] > versions[0], - "Latest delta index should have higher dataset version than original" - ); - - // Now the MemWAL should be trimmed because the delta index was created after the merge - // Our logic should take the maximum dataset version for each index name - trim_mem_wal_index(&mut dataset).await.unwrap(); - - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!( - mem_wal_details.mem_wal_list.len(), - 0, - "MemWAL should be trimmed because delta index was created after flush" - ); + assert_eq!(r1_mg.generation, 10); // Should still be 10 } - #[tokio::test] - async fn test_flush_mem_wal_through_merge_insert() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Add some entries to the MemWAL - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); + #[test] + fn test_empty_merged_generations_noop() { + let mut indices = Vec::new(); - // Seal and flush the MemWAL (required before merging) - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); + // Empty update should be a no-op + update_mem_wal_index_merged_generations(&mut indices, 1, vec![]).unwrap(); - // Verify the MemWAL is flushed but not merged - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Flushed, - "MemWAL should be flushed but not merged yet" - ); - - // Create new data for merge insert - let new_data = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step_custom::(1000, 1)) - .into_df_stream(RowCount::from(100), BatchCount::from(10)); - - // Create merge insert job that will merge the MemWAL - let merge_insert_job = crate::dataset::MergeInsertBuilder::try_new( - Arc::new(dataset.clone()), - vec!["i".to_string()], - ) - .unwrap() - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 0), "owner_0") - .await - .unwrap() - .try_build() - .unwrap(); - - // Execute the merge insert - let (updated_dataset, _stats) = merge_insert_job.execute_reader(new_data).await.unwrap(); - - // Verify that the MemWAL is now marked as merged - let indices = updated_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Merged, - "MemWAL should now be merged" - ); - - // Test that trying to mark a non-existent MemWAL as merged fails - let mut merge_insert_job = crate::dataset::MergeInsertBuilder::try_new( - updated_dataset.clone(), - vec!["i".to_string()], - ) - .unwrap(); - merge_insert_job - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll); - - let result = merge_insert_job - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 999), "owner_0") - .await; - assert!( - result.is_err(), - "Should fail when trying to mark non-existent MemWAL as merged" - ); - - // Test that trying to mark a MemWAL from non-existent region fails - let result = merge_insert_job - .mark_mem_wal_as_merged(MemWalId::new("NONEXISTENT", 0), "owner_0") - .await; - assert!( - result.is_err(), - "Should fail when trying to mark MemWAL from non-existent region as merged" - ); - - // Test that trying to mark an unflushed MemWAL as merged fails - // First, create a new generation that is unsealed - let mut dataset_for_advance = updated_dataset.as_ref().clone(); - advance_mem_wal_generation( - &mut dataset_for_advance, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await - .unwrap(); - - // Update our reference to use the new dataset - let updated_dataset = Arc::new(dataset_for_advance); - - // Verify that generation 1 exists and is unsealed - let indices = updated_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Open, - "Generation 1 should be open" - ); - - let mut merge_insert_job_unsealed = crate::dataset::MergeInsertBuilder::try_new( - updated_dataset.clone(), - vec!["i".to_string()], - ) - .unwrap(); - merge_insert_job_unsealed - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll); - - let result = merge_insert_job_unsealed - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 1), "owner_1") - .await; - assert!( - result.is_err(), - "Should fail when trying to mark unsealed MemWAL as merged" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 1 } is in state Open, but expected Flushed"), - "Error message should indicate the MemWAL is not flushed, got: {}", error); - - // Test that trying to mark an already merged MemWAL as merged fails - let mut merge_insert_job_merged = crate::dataset::MergeInsertBuilder::try_new( - updated_dataset.clone(), - vec!["i".to_string()], - ) - .unwrap(); - merge_insert_job_merged - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll); - - let result = merge_insert_job_merged - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 0), "owner_1") - .await; - assert!( - result.is_err(), - "Should fail when trying to mark already merged MemWAL as merged" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Merged, but expected Flushed"), - "Error message should indicate the MemWAL is already merged, got: {}", error); - - // Test that merge insert with mark_mem_wal_as_merged works correctly when MemWAL is in proper state - // Seal and flush generation 1 and then test the merge insert - let mut dataset_for_seal = updated_dataset.as_ref().clone(); - mark_mem_wal_as_sealed(&mut dataset_for_seal, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - mark_mem_wal_as_flushed(&mut dataset_for_seal, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - let updated_dataset = Arc::new(dataset_for_seal); - - // Verify generation 1 is now flushed but not merged - let indices = updated_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Flushed, - "Generation 1 should be flushed" - ); - - // Create merge insert that merges generation 1 - let new_data_valid = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step_custom::(4000, 1)) - .into_df_stream(RowCount::from(75), BatchCount::from(5)); - - let merge_insert_job_valid = crate::dataset::MergeInsertBuilder::try_new( - updated_dataset.clone(), - vec!["i".to_string()], - ) - .unwrap() - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 1), "owner_1") - .await - .unwrap() - .try_build() - .unwrap(); - - // Execute the merge insert - this should succeed - let (final_dataset, _stats) = merge_insert_job_valid - .execute_reader(new_data_valid) - .await - .unwrap(); - - // Verify that the MemWAL is now marked as merged - let indices = final_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should still exist"); - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Merged, - "Generation 1 should now be merged" - ); - } - - #[tokio::test] - async fn test_replay_mem_wal_with_split_brain_writer() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Add some entries to the MemWAL - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); - - // Simulate a network partition scenario where another node starts replay - // This changes the MemTable location from "mem_table_location_0" to "new_mem_table_location" - update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 0, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await - .unwrap(); - - // Verify the MemTable location was updated - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.mem_table_location, "new_mem_table_location", - "MemTable location should be updated after replay" - ); - - // Now simulate a split-brain scenario where the original writer (node A) - // tries to perform operations using the old MemTable location - - // Test 1: append_mem_wal_entry with old owner_id should fail - let result = append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 789, "owner_0").await; - assert!( - result.is_err(), - "Should fail when using old owner_id for append" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } has owner_id: new_owner_id, but expected owner_0"), - "Error message should indicate owner_id mismatch, got: {}", error); - - // Test 2: mark_mem_wal_as_sealed with old owner_id should fail - let result = mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when using old owner_id for seal" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } has owner_id: new_owner_id, but expected owner_0"), - "Error message should indicate owner_id mismatch, got: {}", error); - - // Test 3: mark_mem_wal_as_flushed with old owner_id should fail - // First seal the MemWAL using the correct owner_id - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "new_owner_id") - .await - .unwrap(); - - let result = mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when using old owner_id for flush" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } has owner_id: new_owner_id, but expected owner_0"), - "Error message should indicate owner_id mismatch, got: {}", error); - - // Test 4: advance_mem_wal_generation with old owner_id should fail - let result = advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), // Using old owner_id - "owner_1", - ) - .await; - assert!( - result.is_err(), - "Should fail when using old owner_id for advance generation" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } has owner_id: new_owner_id, but expected owner_0"), - "Error message should indicate owner_id mismatch, got: {}", error); - - // Test 5: merge_insert with mark_mem_wal_as_merged using old owner_id should fail - // First flush the MemWAL using the correct owner_id so it's ready for merging - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "new_owner_id") - .await - .unwrap(); - - // Try to create merge insert job that merges using the old owner_id - let mut merge_insert_job_builder = crate::dataset::MergeInsertBuilder::try_new( - Arc::new(dataset.clone()), - vec!["i".to_string()], - ) - .unwrap(); - - let build_result = merge_insert_job_builder - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 0), "owner_0") // Using old owner_id - .await; - - assert!( - build_result.is_err(), - "Should fail when using old owner_id for merge insert merge" - ); - - // Check the specific error message - let error = build_result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } has owner_id: new_owner_id, but expected owner_0"), - "Error message should indicate owner_id mismatch for merge insert, got: {}", error); - } - - #[tokio::test] - async fn test_concurrent_mem_wal_replay_and_modifications() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Add some entries to the MemWAL - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); - - // Clone the dataset multiple times to simulate concurrent operations - let mut dataset_clone_append = dataset.clone(); - let mut dataset_clone_seal = dataset.clone(); - let mut dataset_clone_flush = dataset.clone(); - let mut dataset_clone_advance = dataset.clone(); - - // Start replay operation on the original dataset - let replay_result = update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 0, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await; - - // Test all concurrent operations against the replay - let append_result = - append_mem_wal_entry(&mut dataset_clone_append, "GLOBAL", 0, 789, "owner_0").await; - let seal_result = - mark_mem_wal_as_sealed(&mut dataset_clone_seal, "GLOBAL", 0, "owner_0").await; - let flush_result = - mark_mem_wal_as_flushed(&mut dataset_clone_flush, "GLOBAL", 0, "owner_0").await; - let advance_result = advance_mem_wal_generation( - &mut dataset_clone_advance, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await; - - // Test merge_insert merge operation separately (requires flushed MemWAL) - // Advance to a new generation and seal it for merge insert test - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("new_owner_id"), - "owner_1", - ) - .await - .unwrap(); - - // Seal and flush the new generation - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - - let dataset_clone_merge_insert = dataset.clone(); - - // Start replay operation on the new generation - let replay_result_merge_insert = update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 1, - "new_owner_id", - Some("new_mem_table_location_merge"), - ) - .await; - - // Test merge_insert merge operation - let mut merge_insert_job_builder = crate::dataset::MergeInsertBuilder::try_new( - Arc::new(dataset_clone_merge_insert), - vec!["i".to_string()], - ) - .unwrap(); - - let merge_insert_job = merge_insert_job_builder - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 1), "owner_1") - .await - .unwrap() - .try_build() - .unwrap(); - - // Create some data for the merge insert - let new_data = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step_custom::(2000, 1)) - .into_df_stream(RowCount::from(50), BatchCount::from(5)); - - // Execute the merge insert (this should fail due to version conflict) - let merge_insert_result = merge_insert_job.execute_reader(new_data).await; - - // Replay should succeed and all other operations should fail due to version conflict - assert!(replay_result.is_ok(), "Replay operation should succeed"); - assert!( - append_result.is_err(), - "Append operation should fail due to version conflict" - ); - assert!( - seal_result.is_err(), - "Seal operation should fail due to version conflict" - ); - assert!( - flush_result.is_err(), - "Flush operation should fail due to version conflict" - ); - assert!( - advance_result.is_err(), - "Advance generation operation should fail due to version conflict" - ); - - // For merge insert test, replay should succeed and merge insert should fail - assert!( - replay_result_merge_insert.is_ok(), - "Replay operation for merge insert test should succeed" - ); - assert!( - merge_insert_result.is_err(), - "Merge insert flush operation should fail due to version conflict" - ); - } - - #[tokio::test] - async fn test_concurrent_mem_wal_append_and_merge_insert_flush() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Add some entries to generation 0 - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); - - // Seal and flush generation 0 (required for merge insert merge) - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Advance to generation 1 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await - .unwrap(); - - // Add some entries to generation 1 - append_mem_wal_entry(&mut dataset, "GLOBAL", 1, 789, "owner_1") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 1, 790, "owner_1") - .await - .unwrap(); - - // Clone the dataset to simulate concurrent operations - let mut dataset_clone_append = dataset.clone(); - let dataset_clone_merge_insert = dataset.clone(); - - // Test concurrent operations: append to generation 1 and merge_insert merge generation 0 - let append_result = - append_mem_wal_entry(&mut dataset_clone_append, "GLOBAL", 1, 791, "owner_1").await; - - // Create merge insert job that merges generation 0 - let mut merge_insert_job_builder = crate::dataset::MergeInsertBuilder::try_new( - Arc::new(dataset_clone_merge_insert), - vec!["i".to_string()], - ) - .unwrap(); - - let merge_insert_job = merge_insert_job_builder - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 0), "owner_0") - .await - .unwrap() - .try_build() - .unwrap(); - - // Create some data for the merge insert - let new_data = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step_custom::(2000, 1)) - .into_df_stream(RowCount::from(50), BatchCount::from(5)); - - // Execute the merge insert - let merge_insert_result = merge_insert_job.execute_reader(new_data).await; - - // Both operations should succeed since they operate on different generations - assert!( - append_result.is_ok(), - "Append to generation 1 should succeed" - ); - assert!( - merge_insert_result.is_ok(), - "Merge insert flush of generation 0 should succeed" - ); - - // Get the updated dataset from the merge insert result - let (updated_dataset, _stats) = merge_insert_result.unwrap(); - - // Verify the final state using the updated dataset - let indices = updated_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - - // Find generation 0 and generation 1 - let gen_0 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 0) - .expect("Generation 0 should exist"); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - - // Verify generation 0 is merged (after merge_insert) - assert_eq!( - gen_0.state, - lance_index::mem_wal::State::Merged, - "Generation 0 should be merged" - ); - - // Verify generation 1 is unsealed and unflushed - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Open, - "Generation 1 should be open" - ); - - // Verify that generation 1 has the new entry - let wal_entries = gen_1.wal_entries(); - assert!( - wal_entries.contains(791), - "Generation 1 should contain the new entry 791" - ); - } - - #[tokio::test] - async fn test_concurrent_mem_wal_advance_and_merge_insert_flush() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Add some entries to generation 0 - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); - - // Seal and flush generation 0 (required for merge insert merge) - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Advance to generation 1 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await - .unwrap(); - - // Add some entries to generation 1 - append_mem_wal_entry(&mut dataset, "GLOBAL", 1, 789, "owner_1") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 1, 790, "owner_1") - .await - .unwrap(); - - // Clone the dataset to simulate concurrent operations - let mut dataset_clone_advance = dataset.clone(); - let dataset_clone_merge_insert = dataset.clone(); - - // Test concurrent operations: advance to generation 2 and merge_insert flush generation 0 - let advance_result = advance_mem_wal_generation( - &mut dataset_clone_advance, - "GLOBAL", - "mem_table_location_2", - "wal_location_2", - Some("owner_1"), - "owner_2", - ) - .await; - - // Create merge insert job that merges generation 0 - let mut merge_insert_job_builder = crate::dataset::MergeInsertBuilder::try_new( - Arc::new(dataset_clone_merge_insert), - vec!["i".to_string()], - ) - .unwrap(); - - let merge_insert_job = merge_insert_job_builder - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 0), "owner_0") - .await - .unwrap() - .try_build() - .unwrap(); - - // Create some data for the merge insert - let new_data = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step_custom::(2000, 1)) - .into_df_stream(RowCount::from(50), BatchCount::from(5)); - - // Execute the merge insert - let merge_insert_result = merge_insert_job.execute_reader(new_data).await; - - // Both operations should succeed since they operate on different generations - assert!( - advance_result.is_ok(), - "Advance to generation 2 should succeed" - ); - assert!( - merge_insert_result.is_ok(), - "Merge insert flush of generation 0 should succeed" - ); - - // Get the updated dataset from the merge insert result - let (updated_dataset, _stats) = merge_insert_result.unwrap(); - - // Verify the final state using the updated dataset - let indices = updated_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - - // Find all generations - let gen_0 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 0) - .expect("Generation 0 should exist"); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - let gen_2 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 2) - .expect("Generation 2 should exist"); - - // Verify generation 0 is merged (after merge_insert) - assert_eq!( - gen_0.state, - lance_index::mem_wal::State::Merged, - "Generation 0 should be merged" - ); - - // Verify generation 1 is sealed (due to advance) but unflushed - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Sealed, - "Generation 1 should be sealed due to advance" - ); - - // Verify generation 2 is unsealed and unflushed - assert_eq!( - gen_2.state, - lance_index::mem_wal::State::Open, - "Generation 2 should be open" - ); - - // Verify that generation 1 has the expected entries - let wal_entries = gen_1.wal_entries(); - assert!( - wal_entries.contains(789), - "Generation 1 should contain entry 789" - ); - assert!( - wal_entries.contains(790), - "Generation 1 should contain entry 790" - ); + assert!(indices.is_empty()); } } diff --git a/rust/lance/src/io/commit/conflict_resolver.rs b/rust/lance/src/io/commit/conflict_resolver.rs index 0fd52ab2a8b..f65d7047923 100644 --- a/rust/lance/src/io/commit/conflict_resolver.rs +++ b/rust/lance/src/io/commit/conflict_resolver.rs @@ -2,6 +2,7 @@ // SPDX-FileCopyrightText: Copyright The Lance Authors use crate::index::frag_reuse::{build_frag_reuse_index_metadata, load_frag_reuse_index_details}; +use crate::index::mem_wal::{load_mem_wal_index_details, new_mem_wal_index_meta}; use crate::io::deletion::read_dataset_deletion_file; use crate::{ dataset::transaction::{Operation, Transaction}, @@ -13,7 +14,7 @@ use lance_core::{ Error, Result, }; use lance_index::frag_reuse::FRAG_REUSE_INDEX_NAME; -use lance_index::mem_wal::MemWal; +use lance_index::mem_wal::{MergedGeneration, MEM_WAL_INDEX_NAME}; use lance_table::format::IndexMetadata; use lance_table::{format::Fragment, io::deletion::write_deletion_file}; use snafu::{location, Location}; @@ -33,6 +34,9 @@ pub struct TransactionRebase<'a> { modified_fragment_ids: HashSet, affected_rows: Option<&'a RowAddrTreeMap>, conflicting_frag_reuse_indices: Vec, + /// Merged generations from conflicting UpdateMemWalState transactions. + /// Used when rebasing CreateIndex of MemWalIndex. + conflicting_mem_wal_merged_gens: Vec, } impl<'a> TransactionRebase<'a> { @@ -58,6 +62,7 @@ impl<'a> TransactionRebase<'a> { initial_fragments: HashMap::new(), modified_fragment_ids: HashSet::new(), conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }), Operation::Delete { updated_fragments, @@ -85,6 +90,7 @@ impl<'a> TransactionRebase<'a> { modified_fragment_ids, affected_rows: None, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }); } @@ -97,6 +103,7 @@ impl<'a> TransactionRebase<'a> { initial_fragments, modified_fragment_ids, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }) } Operation::Rewrite { groups, .. } => { @@ -114,6 +121,7 @@ impl<'a> TransactionRebase<'a> { initial_fragments, modified_fragment_ids, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }) } Operation::DataReplacement { replacements } => { @@ -128,6 +136,7 @@ impl<'a> TransactionRebase<'a> { initial_fragments, modified_fragment_ids, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }) } Operation::Merge { fragments, .. } => { @@ -141,6 +150,7 @@ impl<'a> TransactionRebase<'a> { initial_fragments, modified_fragment_ids, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }) } } @@ -343,8 +353,8 @@ impl<'a> TransactionRebase<'a> { other_version: u64, ) -> Result<()> { if let Operation::Update { - mem_wal_to_merge, inserted_rows_filter: self_inserted_rows_filter, + merged_generations: self_merged_generations, .. } = &self.transaction.operation { @@ -515,21 +525,14 @@ impl<'a> TransactionRebase<'a> { Operation::Overwrite { .. } | Operation::Restore { .. } => Err( self.incompatible_conflict_err(other_transaction, other_version, location!()) ), - Operation::UpdateMemWalState { added, updated, .. } => { - self.check_update_mem_wal_state_not_modify_same_mem_wal( - added, - mem_wal_to_merge.as_slice(), - other_transaction, - other_version, - )?; - self.check_update_mem_wal_state_not_modify_same_mem_wal( - updated, - mem_wal_to_merge.as_slice(), - other_transaction, - other_version, - )?; - Ok(()) - } + Operation::UpdateMemWalState { + merged_generations: other_merged_generations, + } => self.check_merged_generations_conflict( + other_merged_generations, + self_merged_generations, + other_transaction, + other_version, + ), } } else { Err(wrong_operation_err(&self.transaction.operation)) @@ -552,17 +555,25 @@ impl<'a> TransactionRebase<'a> { | Operation::Clone { .. } | Operation::UpdateBases { .. } => Ok(()), // Indices are identified by UUIDs, so they shouldn't conflict. - // unless it is the same frag reuse index + // unless it is the same frag reuse index or MemWAL index Operation::CreateIndex { new_indices: created_indices, .. } => { - if new_indices + let self_has_frag_reuse = new_indices .iter() - .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME) - && created_indices - .iter() - .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME) + .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME); + let other_has_frag_reuse = created_indices + .iter() + .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME); + let self_has_mem_wal = + new_indices.iter().any(|idx| idx.name == MEM_WAL_INDEX_NAME); + let other_has_mem_wal = created_indices + .iter() + .any(|idx| idx.name == MEM_WAL_INDEX_NAME); + + if (self_has_frag_reuse && other_has_frag_reuse) + || (self_has_mem_wal && other_has_mem_wal) { Err(self.retryable_conflict_err( other_transaction, @@ -664,13 +675,27 @@ impl<'a> TransactionRebase<'a> { } Ok(()) } - Operation::Overwrite { .. } - | Operation::Restore { .. } - | Operation::UpdateMemWalState { .. } => Err(self.incompatible_conflict_err( - other_transaction, - other_version, - location!(), - )), + Operation::UpdateMemWalState { + merged_generations: other_merged_gens, + } => { + // CreateIndex of MemWalIndex is compatible with UpdateMemWalState + // as they can be rebased on each other + if new_indices.iter().any(|idx| idx.name == MEM_WAL_INDEX_NAME) { + // Collect merged_generations from UpdateMemWalState for rebasing + self.conflicting_mem_wal_merged_gens + .extend(other_merged_gens.iter().cloned()); + Ok(()) + } else { + Err(self.incompatible_conflict_err( + other_transaction, + other_version, + location!(), + )) + } + } + Operation::Overwrite { .. } | Operation::Restore { .. } => Err( + self.incompatible_conflict_err(other_transaction, other_version, location!()) + ), } } else { Err(wrong_operation_err(&self.transaction.operation)) @@ -1205,72 +1230,53 @@ impl<'a> TransactionRebase<'a> { other_version: u64, ) -> Result<()> { if let Operation::UpdateMemWalState { - added, - updated, - removed: _, - .. + merged_generations: self_merged_generations, } = &self.transaction.operation { match &other_transaction.operation { Operation::UpdateMemWalState { - added: committed_added, - updated: committed_updated, - removed: _, + merged_generations: other_merged_generations, } => { - // 1. if the current or last committed job is trimming flushed MemWALs, - // it is compatible with any other UpdateMemWalState commits - if (committed_added.is_empty() && committed_updated.is_empty()) - || (added.is_empty() && updated.is_empty()) - { - return Ok(()); - } - - // 2. MemWALs of different regions can be changed at the same time - self.check_update_mem_wal_state_not_modify_same_mem_wal( - committed_added, - added, - other_transaction, - other_version, - )?; - self.check_update_mem_wal_state_not_modify_same_mem_wal( - committed_added, - updated, - other_transaction, - other_version, - )?; - self.check_update_mem_wal_state_not_modify_same_mem_wal( - committed_updated, - added, + // Two UpdateMemWalState transactions conflict if they're updating + // the same region's merged_generation + self.check_merged_generations_conflict( + other_merged_generations, + self_merged_generations, other_transaction, other_version, - )?; - self.check_update_mem_wal_state_not_modify_same_mem_wal( - committed_updated, - updated, - other_transaction, - other_version, - )?; - Ok(()) + ) } Operation::Update { - mem_wal_to_merge, .. + merged_generations: other_merged_generations, + .. } => { - if mem_wal_to_merge.is_some() { - // TODO: This check could be more detailed, there is an assumption that - // once a MemWAL is sealed, there is no other operation that could change - // the state back to open, and at that point it can always be flushed. - Ok(()) - } else { - Err(self.incompatible_conflict_err( + // Update transactions with merged_generations can conflict + self.check_merged_generations_conflict( + other_merged_generations, + self_merged_generations, + other_transaction, + other_version, + ) + } + Operation::CreateIndex { new_indices, .. } => { + // Check if CreateIndex has a MemWalIndex with merged_generations + if let Some(mem_wal_idx) = new_indices + .iter() + .find(|idx| idx.name == MEM_WAL_INDEX_NAME) + { + let details = load_mem_wal_index_details(mem_wal_idx.clone())?; + self.check_merged_generations_conflict( + &details.merged_generations, + self_merged_generations, other_transaction, other_version, - location!(), - )) + ) + } else { + Ok(()) } } Operation::UpdateConfig { .. } | Operation::Rewrite { .. } - | Operation::CreateIndex { .. } | Operation::ReserveFragments { .. } | Operation::UpdateBases { .. } => Ok(()), Operation::Append { .. } @@ -1343,50 +1349,36 @@ impl<'a> TransactionRebase<'a> { } } - fn check_update_mem_wal_state_not_modify_same_mem_wal( + fn check_merged_generations_conflict( &self, - committed: &[MemWal], - to_commit: &[MemWal], + committed: &[MergedGeneration], + to_commit: &[MergedGeneration], other_transaction: &Transaction, other_version: u64, ) -> Result<()> { - if !committed.is_empty() { - if to_commit.is_empty() { - return Ok(()); - } - - if committed.len() > 1 { - return Err(Error::Internal { - message: format!( - "Committing multiple MemWALs is not supported, but found committed: {:?}", - committed - ), - location: location!(), - }); - } - - if to_commit.len() > 1 { - return Err(Error::NotSupported { - source: format!( - "Committing multiple MemWALs is not supported, but found attempt to commit: {:?}", - to_commit - ) - .into(), - location: location!(), - }); - } - - let committed_mem_wal = committed.first().unwrap(); - let to_commit_mem_wal = to_commit.first().unwrap(); - if committed_mem_wal.id == to_commit_mem_wal.id { - return Err(self.incompatible_conflict_err( - other_transaction, - other_version, - location!(), - )); + // Check if any region has conflicting updates + for committed_mg in committed { + for to_commit_mg in to_commit { + if committed_mg.region_id == to_commit_mg.region_id { + // Same region being updated + // If committed >= to_commit, data already merged or superseded - abort without retry + // If committed < to_commit, can retry with new state + if committed_mg.generation >= to_commit_mg.generation { + return Err(self.incompatible_conflict_err( + other_transaction, + other_version, + location!(), + )); + } else { + return Err(self.retryable_conflict_err( + other_transaction, + other_version, + location!(), + )); + } + } } } - Ok(()) } @@ -1582,57 +1574,87 @@ impl<'a> TransactionRebase<'a> { async fn finish_create_index(mut self, dataset: &Dataset) -> Result { if let Operation::CreateIndex { new_indices, .. } = &mut self.transaction.operation { - if !new_indices + // Handle FRAG_REUSE_INDEX rebasing + let has_frag_reuse = new_indices .iter() - .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME) - { - return Ok(self.transaction); - } - - if self.conflicting_frag_reuse_indices.is_empty() { - return Ok(self.transaction); - } + .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME); + + if has_frag_reuse && !self.conflicting_frag_reuse_indices.is_empty() { + // had at least 1 previous rewrite conflict + // get the max reuse version from each run to be added to the cleaned up index + let mut max_versions = + Vec::with_capacity(self.conflicting_frag_reuse_indices.len()); + for committed_fri in &self.conflicting_frag_reuse_indices { + let committed_fri_details = Arc::try_unwrap( + load_frag_reuse_index_details(dataset, committed_fri) + .await + .unwrap(), + ) + .unwrap(); + let max_version = committed_fri_details + .versions + .into_iter() + .max_by_key(|v| v.dataset_version) + .unwrap(); + max_versions.push(max_version); + } - // had at least 1 previous rewrite conflict - // get the max reuse version from each run to be added to the cleaned up index - let mut max_versions = Vec::with_capacity(self.conflicting_frag_reuse_indices.len()); - for committed_fri in &self.conflicting_frag_reuse_indices { - let committed_fri_details = Arc::try_unwrap( - load_frag_reuse_index_details(dataset, committed_fri) + // there should be only 1 frag_reuse_index in new indices + let new_fri = &new_indices[0]; + let mut new_fri_details = Arc::try_unwrap( + load_frag_reuse_index_details(dataset, new_fri) .await .unwrap(), ) .unwrap(); - let max_version = committed_fri_details - .versions - .into_iter() - .max_by_key(|v| v.dataset_version) - .unwrap(); - max_versions.push(max_version); + new_fri_details.versions.extend(max_versions); + + let new_frag_bitmap = new_fri_details.new_frag_bitmap(); + + let new_frag_reuse_index_meta = build_frag_reuse_index_metadata( + dataset, + Some(new_fri), + new_fri_details, + new_frag_bitmap, + ) + .await?; + + new_indices.retain(|idx| idx.name != FRAG_REUSE_INDEX_NAME); + new_indices.push(new_frag_reuse_index_meta); } - // there should be only 1 frag_reuse_index in new indices - let new_fri = &new_indices[0]; - let mut new_fri_details = Arc::try_unwrap( - load_frag_reuse_index_details(dataset, new_fri) - .await - .unwrap(), - ) - .unwrap(); - new_fri_details.versions.extend(max_versions); + // Handle MEM_WAL_INDEX rebasing + let has_mem_wal = new_indices.iter().any(|idx| idx.name == MEM_WAL_INDEX_NAME); - let new_frag_bitmap = new_fri_details.new_frag_bitmap(); + if has_mem_wal && !self.conflicting_mem_wal_merged_gens.is_empty() { + let pos = new_indices + .iter() + .position(|idx| idx.name == MEM_WAL_INDEX_NAME) + .unwrap(); - let new_frag_reuse_index_meta = build_frag_reuse_index_metadata( - dataset, - Some(new_fri), - new_fri_details, - new_frag_bitmap, - ) - .await?; + let current_meta = new_indices.remove(pos); + let mut details = load_mem_wal_index_details(current_meta)?; + + // Merge conflicting merged_generations - for each region, keep higher generation + // We own self so we can consume conflicting_mem_wal_merged_gens directly + for new_mg in self.conflicting_mem_wal_merged_gens { + if let Some(existing) = details + .merged_generations + .iter_mut() + .find(|mg| mg.region_id == new_mg.region_id) + { + if new_mg.generation > existing.generation { + existing.generation = new_mg.generation; + } + } else { + details.merged_generations.push(new_mg); + } + } + + let new_meta = new_mem_wal_index_meta(dataset.manifest.version, details)?; + new_indices.push(new_meta); + } - new_indices.retain(|idx| idx.name != FRAG_REUSE_INDEX_NAME); - new_indices.push(new_frag_reuse_index_meta); Ok(self.transaction) } else { Err(wrong_operation_err(&self.transaction.operation)) @@ -1767,6 +1789,7 @@ mod tests { use lance_core::Error; use lance_file::version::LanceFileVersion; use lance_io::assert_io_eq; + use uuid::Uuid; use lance_table::format::IndexMetadata; use lance_table::io::deletion::{deletion_file_path, read_deletion_file}; @@ -1861,7 +1884,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -1873,7 +1896,7 @@ mod tests { removed_fragment_ids: vec![2], new_fragments: vec![], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -1888,7 +1911,7 @@ mod tests { updated_fragments: vec![Fragment::new(4)], new_fragments: vec![], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -1990,7 +2013,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![sample_file.clone()], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2005,7 +2028,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![sample_file], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2127,7 +2150,7 @@ mod tests { removed_fragment_ids: vec![0], new_fragments: vec![sample_file.clone()], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2140,7 +2163,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![sample_file.clone()], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2299,7 +2322,7 @@ mod tests { updated_fragments: vec![fragment0.clone()], new_fragments: vec![fragment2.clone()], fields_modified: vec![0], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2495,7 +2518,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![fragment2], fields_modified: vec![0], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2688,6 +2711,7 @@ mod tests { modified_fragment_ids: modified_fragment_ids(operation).collect::>(), affected_rows: None, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }; for (other, expected_conflict) in other_transactions.iter().zip(expected_conflicts) { @@ -2918,7 +2942,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -3208,6 +3232,7 @@ mod tests { modified_fragment_ids: modified_fragment_ids(&op1).collect::>(), affected_rows: None, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }; let result = rebase.check_txn(&txn2, 1); @@ -3239,4 +3264,287 @@ mod tests { } } } + + #[test] + fn test_merged_generations_conflict_lower_generation_fails() { + // Test: committed generation >= to_commit generation should be incompatible (no retry) + let region = Uuid::new_v4(); + + // Committed has generation 10, we're trying to commit generation 5 + let committed_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + + let to_commit_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result = rebase.check_txn(&committed_txn, 1); + assert!( + matches!(result, Err(Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict for lower generation, got {:?}", + result + ); + } + + #[test] + fn test_merged_generations_conflict_equal_generation_fails() { + // Test: committed generation == to_commit generation should be incompatible (no retry) + let region = Uuid::new_v4(); + + let committed_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + + let to_commit_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result = rebase.check_txn(&committed_txn, 1); + assert!( + matches!(result, Err(Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict for equal generation, got {:?}", + result + ); + } + + #[test] + fn test_merged_generations_conflict_higher_generation_retryable() { + // Test: committed generation < to_commit generation should be retryable + let region = Uuid::new_v4(); + + // Committed has generation 5, we're trying to commit generation 10 + let committed_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, + None, + ); + + let to_commit_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result = rebase.check_txn(&committed_txn, 1); + assert!( + matches!(result, Err(Error::RetryableCommitConflict { .. })), + "Expected retryable conflict for higher generation, got {:?}", + result + ); + } + + #[test] + fn test_merged_generations_different_regions_ok() { + // Test: different regions should not conflict + let region1 = Uuid::new_v4(); + let region2 = Uuid::new_v4(); + + let committed_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region1, 10)], + }, + None, + ); + + let to_commit_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region2, 5)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result = rebase.check_txn(&committed_txn, 1); + assert!( + result.is_ok(), + "Expected OK for different regions, got {:?}", + result + ); + } + + #[test] + fn test_update_mem_wal_state_vs_create_index_with_merged_generations() { + use crate::index::mem_wal::new_mem_wal_index_meta; + use lance_index::mem_wal::MemWalIndexDetails; + + let region = Uuid::new_v4(); + + // Create a MemWalIndex with merged_generations + let details = MemWalIndexDetails { + merged_generations: vec![MergedGeneration::new(region, 10)], + ..Default::default() + }; + let mem_wal_index = new_mem_wal_index_meta(1, details).unwrap(); + + // CreateIndex with MemWalIndex that has generation 10 + let committed_txn = Transaction::new( + 0, + Operation::CreateIndex { + new_indices: vec![mem_wal_index], + removed_indices: vec![], + }, + None, + ); + + // UpdateMemWalState trying to set generation 5 (lower than committed) + let to_commit_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result = rebase.check_txn(&committed_txn, 1); + assert!( + matches!(result, Err(Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict when UpdateMemWalState generation is lower than CreateIndex, got {:?}", + result + ); + + // Now test with higher generation (should be retryable) + let to_commit_txn_higher = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 15)], + }, + None, + ); + + let mut rebase_higher = TransactionRebase { + transaction: to_commit_txn_higher, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result_higher = rebase_higher.check_txn(&committed_txn, 1); + assert!( + matches!(result_higher, Err(Error::RetryableCommitConflict { .. })), + "Expected retryable conflict when UpdateMemWalState generation is higher than CreateIndex, got {:?}", + result_higher + ); + } + + #[test] + fn test_create_index_vs_update_mem_wal_state_rebase() { + use crate::index::mem_wal::new_mem_wal_index_meta; + use lance_index::mem_wal::MemWalIndexDetails; + + let region = Uuid::new_v4(); + + // CreateIndex with MemWalIndex (no merged_generations initially) + let details = MemWalIndexDetails::default(); + let mem_wal_index = new_mem_wal_index_meta(1, details).unwrap(); + + let to_commit_txn = Transaction::new( + 0, + Operation::CreateIndex { + new_indices: vec![mem_wal_index], + removed_indices: vec![], + }, + None, + ); + + // UpdateMemWalState with generation 10 + let committed_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + // CreateIndex of MemWalIndex should be compatible with UpdateMemWalState + // and should collect the merged_generations for rebasing + let result = rebase.check_txn(&committed_txn, 1); + assert!( + result.is_ok(), + "Expected OK for CreateIndex vs UpdateMemWalState, got {:?}", + result + ); + + // Verify that merged_generations were collected + assert_eq!(rebase.conflicting_mem_wal_merged_gens.len(), 1); + assert_eq!(rebase.conflicting_mem_wal_merged_gens[0].region_id, region); + assert_eq!(rebase.conflicting_mem_wal_merged_gens[0].generation, 10); + } }