From 57eb63cee06f623585f56c8d494cf91f7d3c2698 Mon Sep 17 00:00:00 2001 From: Pere Diaz Bou Date: Thu, 16 Oct 2025 14:50:08 +0200 Subject: [PATCH] core/bree: remove duplicated code in BTreeCursor --- core/incremental/aggregate_operator.rs | 1 + core/incremental/compiler.rs | 2 +- core/incremental/cursor.rs | 2 +- core/incremental/join_operator.rs | 1 + .../mvcc/database/checkpoint_state_machine.rs | 2 +- core/storage/btree.rs | 716 +----------------- 6 files changed, 34 insertions(+), 690 deletions(-) diff --git a/core/incremental/aggregate_operator.rs b/core/incremental/aggregate_operator.rs index e577f05e2..e0ed53d9f 100644 --- a/core/incremental/aggregate_operator.rs +++ b/core/incremental/aggregate_operator.rs @@ -7,6 +7,7 @@ use crate::incremental::operator::{ generate_storage_id, ComputationTracker, DbspStateCursors, EvalState, IncrementalOperator, }; use crate::incremental::persistence::{ReadRecord, WriteRow}; +use crate::storage::btree::CursorTrait; use crate::types::{IOResult, ImmutableRecord, SeekKey, SeekOp, SeekResult, ValueRef}; use crate::{return_and_restore_if_io, return_if_io, LimboError, Result, Value}; use std::collections::{BTreeMap, HashMap, HashSet}; diff --git a/core/incremental/compiler.rs b/core/incremental/compiler.rs index b62795fab..98cc29896 100644 --- a/core/incremental/compiler.rs +++ b/core/incremental/compiler.rs @@ -12,7 +12,7 @@ use crate::incremental::operator::{ IncrementalOperator, InputOperator, JoinOperator, JoinType, ProjectOperator, }; use crate::schema::Type; -use crate::storage::btree::{BTreeCursor, BTreeKey}; +use crate::storage::btree::{BTreeCursor, BTreeKey, CursorTrait}; // Note: logical module must be made pub(crate) in translate/mod.rs use crate::translate::logical::{ BinaryOperator, Column, ColumnInfo, JoinType as LogicalJoinType, LogicalExpr, LogicalPlan, diff --git a/core/incremental/cursor.rs b/core/incremental/cursor.rs index 22070f29d..a33734c72 100644 --- a/core/incremental/cursor.rs +++ b/core/incremental/cursor.rs @@ -5,7 +5,7 @@ use crate::{ view::{IncrementalView, ViewTransactionState}, }, return_if_io, - storage::btree::BTreeCursor, + storage::btree::{BTreeCursor, CursorTrait}, types::{IOResult, SeekKey, SeekOp, SeekResult, Value}, LimboError, Pager, Result, }; diff --git a/core/incremental/join_operator.rs b/core/incremental/join_operator.rs index 722274559..982545ca9 100644 --- a/core/incremental/join_operator.rs +++ b/core/incremental/join_operator.rs @@ -6,6 +6,7 @@ use crate::incremental::operator::{ generate_storage_id, ComputationTracker, DbspStateCursors, EvalState, IncrementalOperator, }; use crate::incremental::persistence::WriteRow; +use crate::storage::btree::CursorTrait; use crate::types::{IOResult, ImmutableRecord, SeekKey, SeekOp, SeekResult}; use crate::{return_and_restore_if_io, return_if_io, Result, Value}; use std::sync::{Arc, Mutex}; diff --git a/core/mvcc/database/checkpoint_state_machine.rs b/core/mvcc/database/checkpoint_state_machine.rs index 16b5d3a4a..778431adc 100644 --- a/core/mvcc/database/checkpoint_state_machine.rs +++ b/core/mvcc/database/checkpoint_state_machine.rs @@ -4,7 +4,7 @@ use crate::mvcc::database::{ SQLITE_SCHEMA_MVCC_TABLE_ID, }; use crate::state_machine::{StateMachine, StateTransition, TransitionResult}; -use crate::storage::btree::BTreeCursor; +use crate::storage::btree::{BTreeCursor, CursorTrait}; use crate::storage::pager::CreateBTreeFlags; use crate::storage::wal::{CheckpointMode, TursoRwLock}; use crate::types::{IOCompletions, IOResult, ImmutableRecord, RecordCursor}; diff --git a/core/storage/btree.rs b/core/storage/btree.rs index 633c97599..6b1441538 100644 --- a/core/storage/btree.rs +++ b/core/storage/btree.rs @@ -4452,562 +4452,6 @@ impl BTreeCursor { self.usable_space_cached } - #[instrument(skip(self), level = Level::DEBUG)] - pub fn rowid(&self) -> Result>> { - if let Some(mv_cursor) = &self.mv_cursor { - let mv_cursor = mv_cursor.write(); - let IOResult::Done(rowid) = mv_cursor.rowid()? else { - todo!() - }; - let Some(rowid) = rowid else { - return Ok(IOResult::Done(None)); - }; - return Ok(IOResult::Done(Some(rowid))); - } - if self.get_null_flag() { - return Ok(IOResult::Done(None)); - } - if self.has_record.get() { - let page = self.stack.top_ref(); - let contents = page.get_contents(); - let page_type = contents.page_type(); - if page_type.is_table() { - let cell_idx = self.stack.current_cell_index(); - let rowid = contents.cell_table_leaf_read_rowid(cell_idx as usize)?; - Ok(IOResult::Done(Some(rowid))) - } else { - let _ = return_if_io!(self.record()); - Ok(IOResult::Done(self.get_index_rowid_from_record())) - } - } else { - Ok(IOResult::Done(None)) - } - } - - #[instrument(skip(self, key), level = Level::DEBUG)] - pub fn seek(&mut self, key: SeekKey<'_>, op: SeekOp) -> Result> { - if let Some(mv_cursor) = &self.mv_cursor { - let mut mv_cursor = mv_cursor.write(); - return mv_cursor.seek(key, op); - } - self.skip_advance.set(false); - // Empty trace to capture the span information - tracing::trace!(""); - // We need to clear the null flag for the table cursor before seeking, - // because it might have been set to false by an unmatched left-join row during the previous iteration - // on the outer loop. - self.set_null_flag(false); - let seek_result = return_if_io!(self.do_seek(key, op)); - self.invalidate_record(); - // Reset seek state - self.seek_state = CursorSeekState::Start; - self.valid_state = CursorValidState::Valid; - Ok(IOResult::Done(seek_result)) - } - - /// Return a reference to the record the cursor is currently pointing to. - /// If record was not parsed yet, then we have to parse it and in case of I/O we yield control - /// back. - #[instrument(skip(self), level = Level::DEBUG)] - pub fn record(&self) -> Result>>> { - if !self.has_record.get() && self.mv_cursor.is_none() { - return Ok(IOResult::Done(None)); - } - let invalidated = self - .reusable_immutable_record - .borrow() - .as_ref() - .is_none_or(|record| record.is_invalidated()); - if !invalidated { - let record_ref = - Ref::filter_map(self.reusable_immutable_record.borrow(), |opt| opt.as_ref()) - .unwrap(); - return Ok(IOResult::Done(Some(record_ref))); - } - if let Some(mv_cursor) = &self.mv_cursor { - let mv_cursor = mv_cursor.write(); - let Some(row) = mv_cursor.current_row()? else { - return Ok(IOResult::Done(None)); - }; - self.get_immutable_record_or_create() - .as_mut() - .unwrap() - .invalidate(); - self.get_immutable_record_or_create() - .as_mut() - .unwrap() - .start_serialization(&row.data); - self.record_cursor.borrow_mut().invalidate(); - let record_ref = - Ref::filter_map(self.reusable_immutable_record.borrow(), |opt| opt.as_ref()) - .unwrap(); - return Ok(IOResult::Done(Some(record_ref))); - } - - let page = self.stack.top_ref(); - let contents = page.get_contents(); - let cell_idx = self.stack.current_cell_index(); - let cell = contents.cell_get(cell_idx as usize, self.usable_space())?; - let (payload, payload_size, first_overflow_page) = match cell { - BTreeCell::TableLeafCell(TableLeafCell { - payload, - payload_size, - first_overflow_page, - .. - }) => (payload, payload_size, first_overflow_page), - BTreeCell::IndexInteriorCell(IndexInteriorCell { - payload, - payload_size, - first_overflow_page, - .. - }) => (payload, payload_size, first_overflow_page), - BTreeCell::IndexLeafCell(IndexLeafCell { - payload, - first_overflow_page, - payload_size, - }) => (payload, payload_size, first_overflow_page), - _ => unreachable!("unexpected page_type"), - }; - if let Some(next_page) = first_overflow_page { - return_if_io!(self.process_overflow_read(payload, next_page, payload_size)) - } else { - self.get_immutable_record_or_create() - .as_mut() - .unwrap() - .invalidate(); - self.get_immutable_record_or_create() - .as_mut() - .unwrap() - .start_serialization(payload); - self.record_cursor.borrow_mut().invalidate(); - }; - - let record_ref = - Ref::filter_map(self.reusable_immutable_record.borrow(), |opt| opt.as_ref()).unwrap(); - Ok(IOResult::Done(Some(record_ref))) - } - - #[instrument(skip_all, level = Level::DEBUG)] - pub fn insert(&mut self, key: &BTreeKey) -> Result> { - tracing::debug!(valid_state = ?self.valid_state, cursor_state = ?self.state, is_write_in_progress = self.is_write_in_progress()); - match &self.mv_cursor { - Some(mv_cursor) => { - return_if_io!(mv_cursor.write().insert(key)); - } - None => { - return_if_io!(self.insert_into_page(key)); - if key.maybe_rowid().is_some() { - self.has_record.replace(true); - } - } - }; - Ok(IOResult::Done(())) - } - - /// Delete state machine flow: - /// 1. Start -> check if the rowid to be delete is present in the page or not. If not we early return - /// 2. DeterminePostBalancingSeekKey -> determine the key to seek to after balancing. - /// 3. LoadPage -> load the page. - /// 4. FindCell -> find the cell to be deleted in the page. - /// 5. ClearOverflowPages -> Clear the overflow pages if there are any before dropping the cell, then if we are in a leaf page we just drop the cell in place. - /// if we are in interior page, we need to rotate keys in order to replace current cell (InteriorNodeReplacement). - /// 6. InteriorNodeReplacement -> we copy the left subtree leaf node into the deleted interior node's place. - /// 7. Balancing -> perform balancing - /// 8. PostInteriorNodeReplacement -> if an interior node was replaced, we need to advance the cursor once. - /// 9. SeekAfterBalancing -> adjust the cursor to a node that is closer to the deleted value. go to Finish - /// 10. Finish -> Delete operation is done. Return CursorResult(Ok()) - #[instrument(skip(self), level = Level::DEBUG)] - pub fn delete(&mut self) -> Result> { - if let Some(mv_cursor) = &self.mv_cursor { - return_if_io!(mv_cursor.write().delete()); - return Ok(IOResult::Done(())); - } - - if let CursorState::None = &self.state { - self.state = CursorState::Delete(DeleteState::Start); - } - - loop { - let usable_space = self.usable_space(); - let delete_state = match &mut self.state { - CursorState::Delete(x) => x, - _ => unreachable!("expected delete state"), - }; - tracing::debug!(?delete_state); - - match delete_state { - DeleteState::Start => { - let page = self.stack.top_ref(); - self.pager.add_dirty(page); - if matches!( - page.get_contents().page_type(), - PageType::TableLeaf | PageType::TableInterior - ) { - if return_if_io!(self.rowid()).is_none() { - self.state = CursorState::None; - return Ok(IOResult::Done(())); - } - } else if self.reusable_immutable_record.borrow().is_none() { - self.state = CursorState::None; - return Ok(IOResult::Done(())); - } - - self.state = CursorState::Delete(DeleteState::DeterminePostBalancingSeekKey); - } - - DeleteState::DeterminePostBalancingSeekKey => { - // FIXME: skip this work if we determine deletion wont result in balancing - // Right now we calculate the key every time for simplicity/debugging - // since it won't affect correctness which is more important - let page = self.stack.top_ref(); - let target_key = if page.is_index() { - let record = match return_if_io!(self.record()) { - Some(record) => record.clone(), - None => unreachable!("there should've been a record"), - }; - CursorContext { - key: CursorContextKey::IndexKeyRowId(record), - seek_op: SeekOp::GE { eq_only: true }, - } - } else { - let Some(rowid) = return_if_io!(self.rowid()) else { - panic!("cursor should be pointing to a record with a rowid"); - }; - CursorContext { - key: CursorContextKey::TableRowId(rowid), - seek_op: SeekOp::GE { eq_only: true }, - } - }; - - self.state = CursorState::Delete(DeleteState::LoadPage { - post_balancing_seek_key: Some(target_key), - }); - } - - DeleteState::LoadPage { - post_balancing_seek_key, - } => { - self.state = CursorState::Delete(DeleteState::FindCell { - post_balancing_seek_key: post_balancing_seek_key.take(), - }); - } - - DeleteState::FindCell { - post_balancing_seek_key, - } => { - let page = self.stack.top_ref(); - let cell_idx = self.stack.current_cell_index() as usize; - let contents = page.get_contents(); - if cell_idx >= contents.cell_count() { - return_corrupt!(format!( - "Corrupted page: cell index {} is out of bounds for page with {} cells", - cell_idx, - contents.cell_count() - )); - } - - tracing::debug!( - "DeleteState::FindCell: page_id: {}, cell_idx: {}", - page.get().id, - cell_idx - ); - - let cell = contents.cell_get(cell_idx, usable_space)?; - - let original_child_pointer = match &cell { - BTreeCell::TableInteriorCell(interior) => Some(interior.left_child_page), - BTreeCell::IndexInteriorCell(interior) => Some(interior.left_child_page), - _ => None, - }; - - self.state = CursorState::Delete(DeleteState::ClearOverflowPages { - cell_idx, - cell, - original_child_pointer, - post_balancing_seek_key: post_balancing_seek_key.take(), - }); - } - - DeleteState::ClearOverflowPages { cell, .. } => { - let cell = cell.clone(); - return_if_io!(self.clear_overflow_pages(&cell)); - - let CursorState::Delete(DeleteState::ClearOverflowPages { - cell_idx, - original_child_pointer, - ref mut post_balancing_seek_key, - .. - }) = self.state - else { - unreachable!("expected clear overflow pages state"); - }; - - let page = self.stack.top_ref(); - let contents = page.get_contents(); - - if !contents.is_leaf() { - self.state = CursorState::Delete(DeleteState::InteriorNodeReplacement { - page: page.clone(), - btree_depth: self.stack.current(), - cell_idx, - original_child_pointer, - post_balancing_seek_key: post_balancing_seek_key.take(), - }); - } else { - drop_cell(contents, cell_idx, usable_space)?; - - self.state = CursorState::Delete(DeleteState::CheckNeedsBalancing { - btree_depth: self.stack.current(), - post_balancing_seek_key: post_balancing_seek_key.take(), - interior_node_was_replaced: false, - }); - } - } - - DeleteState::InteriorNodeReplacement { .. } => { - // This is an interior node, we need to handle deletion differently. - // 1. Move cursor to the largest key in the left subtree. - // 2. Replace the cell in the interior (parent) node with that key. - // 3. Delete that key from the child page. - - // Step 1: Move cursor to the largest key in the left subtree. - // The largest key is always in a leaf, and so this traversal may involvegoing multiple pages downwards, - // so we store the page we are currently on. - - // avoid calling prev() because it internally calls restore_context() which may cause unintended behavior. - return_if_io!(self.get_prev_record()); - - let CursorState::Delete(DeleteState::InteriorNodeReplacement { - ref page, - btree_depth, - cell_idx, - original_child_pointer, - ref mut post_balancing_seek_key, - .. - }) = self.state - else { - unreachable!("expected interior node replacement state"); - }; - - // Ensure we keep the parent page at the same position as before the replacement. - self.stack - .node_states - .get_mut(btree_depth) - .expect("parent page should be on the stack") - .cell_idx = cell_idx as i32; - let (cell_payload, leaf_cell_idx) = { - let leaf_page = self.stack.top_ref(); - let leaf_contents = leaf_page.get_contents(); - assert!(leaf_contents.is_leaf()); - assert!(leaf_contents.cell_count() > 0); - let leaf_cell_idx = leaf_contents.cell_count() - 1; - let last_cell_on_child_page = - leaf_contents.cell_get(leaf_cell_idx, usable_space)?; - - let mut cell_payload: Vec = Vec::new(); - let child_pointer = - original_child_pointer.expect("there should be a pointer"); - // Rewrite the old leaf cell as an interior cell depending on type. - match last_cell_on_child_page { - BTreeCell::TableLeafCell(leaf_cell) => { - // Table interior cells contain the left child pointer and the rowid as varint. - cell_payload.extend_from_slice(&child_pointer.to_be_bytes()); - write_varint_to_vec(leaf_cell.rowid as u64, &mut cell_payload); - } - BTreeCell::IndexLeafCell(leaf_cell) => { - // Index interior cells contain: - // 1. The left child pointer - // 2. The payload size as varint - // 3. The payload - // 4. The first overflow page as varint, omitted if no overflow. - cell_payload.extend_from_slice(&child_pointer.to_be_bytes()); - write_varint_to_vec(leaf_cell.payload_size, &mut cell_payload); - cell_payload.extend_from_slice(leaf_cell.payload); - if let Some(first_overflow_page) = leaf_cell.first_overflow_page { - cell_payload - .extend_from_slice(&first_overflow_page.to_be_bytes()); - } - } - _ => unreachable!("Expected table leaf cell"), - } - (cell_payload, leaf_cell_idx) - }; - - let leaf_page = self.stack.top_ref(); - - self.pager.add_dirty(page); - self.pager.add_dirty(leaf_page); - - // Step 2: Replace the cell in the parent (interior) page. - { - let parent_contents = page.get_contents(); - let parent_page_id = page.get().id; - let left_child_page = u32::from_be_bytes( - cell_payload[..4].try_into().expect("invalid cell payload"), - ); - turso_assert!( - left_child_page as usize != parent_page_id, - "corrupt: current page and left child page of cell {} are both {}", - left_child_page, - parent_page_id - ); - - // First, drop the old cell that is being replaced. - drop_cell(parent_contents, cell_idx, usable_space)?; - // Then, insert the new cell (the predecessor) in its place. - insert_into_cell(parent_contents, &cell_payload, cell_idx, usable_space)?; - } - - // Step 3: Delete the predecessor cell from the leaf page. - { - let leaf_contents = leaf_page.get_contents(); - drop_cell(leaf_contents, leaf_cell_idx, usable_space)?; - } - - self.state = CursorState::Delete(DeleteState::CheckNeedsBalancing { - btree_depth, - post_balancing_seek_key: post_balancing_seek_key.take(), - interior_node_was_replaced: true, - }); - } - - DeleteState::CheckNeedsBalancing { btree_depth, .. } => { - let page = self.stack.top_ref(); - // Check if either the leaf page we took the replacement cell from underflows, or if the interior page we inserted it into overflows OR underflows. - // If the latter is true, we must always balance that level regardless of whether the leaf page (or any ancestor pages in between) need balancing. - - let leaf_underflows = { - let leaf_contents = page.get_contents(); - let free_space = compute_free_space(leaf_contents, usable_space); - free_space * 3 > usable_space * 2 - }; - - let interior_overflows_or_underflows = { - // Invariant: ancestor pages on the stack are pinned to the page cache, - // so we don't need return_if_locked_maybe_load! any ancestor, - // and we already loaded the current page above. - let interior_page = self - .stack - .get_page_at_level(*btree_depth) - .expect("ancestor page should be on the stack"); - let interior_contents = interior_page.get_contents(); - let overflows = !interior_contents.overflow_cells.is_empty(); - if overflows { - true - } else { - let free_space = compute_free_space(interior_contents, usable_space); - free_space * 3 > usable_space * 2 - } - }; - - let needs_balancing = leaf_underflows || interior_overflows_or_underflows; - - let CursorState::Delete(DeleteState::CheckNeedsBalancing { - btree_depth, - ref mut post_balancing_seek_key, - interior_node_was_replaced, - .. - }) = self.state - else { - unreachable!("expected check needs balancing state"); - }; - - if needs_balancing { - let balance_only_ancestor = - !leaf_underflows && interior_overflows_or_underflows; - if balance_only_ancestor { - // Only need to balance the ancestor page; move there immediately. - while self.stack.current() > btree_depth { - self.stack.pop(); - } - } - let balance_both = leaf_underflows && interior_overflows_or_underflows; - assert!(matches!(self.balance_state.sub_state, BalanceSubState::Start), "There should be no balancing operation in progress when delete state is {:?}, got: {:?}", self.state, self.balance_state.sub_state); - let post_balancing_seek_key = post_balancing_seek_key - .take() - .expect("post_balancing_seek_key should be Some"); - self.save_context(post_balancing_seek_key); - self.state = CursorState::Delete(DeleteState::Balancing { - balance_ancestor_at_depth: if balance_both { - Some(btree_depth) - } else { - None - }, - }); - } else { - // No balancing needed. - if interior_node_was_replaced { - // If we did replace an interior node, we need to advance the cursor once to - // get back at the interior node that now has the replaced content. - // The reason it is important to land here is that the replaced cell was smaller (LT) than the deleted cell, - // so we must ensure we skip over it. I.e., when BTreeCursor::next() is called, it will move past the cell - // that holds the replaced content. - self.state = - CursorState::Delete(DeleteState::PostInteriorNodeReplacement); - } else { - // If we didn't replace an interior node, we are done, - // except we need to retreat, so that the next call to BTreeCursor::next() lands at the next record (because we deleted the current one) - self.stack.retreat(); - self.state = CursorState::None; - return Ok(IOResult::Done(())); - } - } - } - DeleteState::PostInteriorNodeReplacement => { - return_if_io!(self.get_next_record()); - self.state = CursorState::None; - return Ok(IOResult::Done(())); - } - - DeleteState::Balancing { - balance_ancestor_at_depth, - } => { - let balance_ancestor_at_depth = *balance_ancestor_at_depth; - return_if_io!(self.balance(balance_ancestor_at_depth)); - self.state = CursorState::Delete(DeleteState::RestoreContextAfterBalancing); - } - DeleteState::RestoreContextAfterBalancing => { - return_if_io!(self.restore_context()); - - // We deleted key K, and performed a seek to: GE { eq_only: true } K. - // This means that the cursor is now pointing to the next key after K. - // We need to make the next call to BTreeCursor::next() a no-op so that we don't skip over - // a row when deleting rows in a loop. - self.skip_advance.set(true); - self.state = CursorState::None; - return Ok(IOResult::Done(())); - } - } - } - } - - /// In outer joins, whenever the right-side table has no matching row, the query must still return a row - /// for each left-side row. In order to achieve this, we set the null flag on the right-side table cursor - /// so that it returns NULL for all columns until cleared. - #[inline(always)] - pub fn set_null_flag(&mut self, flag: bool) { - self.null_flag = flag; - } - - #[inline(always)] - pub fn get_null_flag(&self) -> bool { - self.null_flag - } - - #[instrument(skip_all, level = Level::DEBUG)] - pub fn exists(&mut self, key: &Value) -> Result> { - assert!(self.mv_cursor.is_none()); - let int_key = match key { - Value::Integer(i) => i, - _ => unreachable!("btree tables are indexed by integers!"), - }; - let seek_result = - return_if_io!(self.seek(SeekKey::TableRowId(*int_key), SeekOp::GE { eq_only: true })); - let exists = matches!(seek_result, SeekResult::Found); - self.invalidate_record(); - Ok(IOResult::Done(exists)) - } - /// Clear the overflow pages linked to a specific page provided by the leaf cell /// Uses a state machine to keep track of it's operations so that traversal can be /// resumed from last point after IO interruption @@ -5088,26 +4532,6 @@ impl BTreeCursor { } } - /// Deletes all content from the B-Tree but preserves the root page. - /// - /// Unlike [`btree_destroy`], which frees all pages including the root, - /// this method only clears the tree’s contents. The root page remains - /// allocated and is reset to an empty leaf page. - pub fn clear_btree(&mut self) -> Result>> { - self.destroy_btree_contents(true) - } - - /// Destroys the entire B-Tree, including the root page. - /// - /// All pages belonging to the tree are freed, leaving no trace of the B-Tree. - /// Use this when the structure itself is no longer needed. - /// - /// For cases where the B-Tree should remain allocated but emptied, see [`btree_clear`]. - #[instrument(skip(self), level = Level::DEBUG)] - pub fn btree_destroy(&mut self) -> Result>> { - self.destroy_btree_contents(false) - } - /// Deletes all contents of the B-tree by freeing all its pages in an iterative depth-first order. /// This ensures child pages are freed before their parents /// Uses a state machine to keep track of the operation to ensure IO doesn't cause repeated traversals @@ -5431,117 +4855,6 @@ impl BTreeCursor { matches!(self.state, CursorState::Write(_)) } - /// Count the number of entries in the b-tree - /// - /// Only supposed to be used in the context of a simple Count Select Statement - #[instrument(skip(self), level = Level::DEBUG)] - pub fn count(&mut self) -> Result> { - if let Some(_mv_cursor) = &self.mv_cursor { - todo!("Implement count for mvcc"); - } - - let mut mem_page; - let mut contents; - - 'outer: loop { - let state = self.count_state; - match state { - CountState::Start => { - let c = self.move_to_root()?; - self.count_state = CountState::Loop; - if let Some(c) = c { - io_yield_one!(c); - } - } - CountState::Loop => { - self.stack.advance(); - mem_page = self.stack.top_ref(); - contents = mem_page.get_contents(); - - /* If this is a leaf page or the tree is not an int-key tree, then - ** this page contains countable entries. Increment the entry counter - ** accordingly. - */ - if !matches!(contents.page_type(), PageType::TableInterior) { - self.count += contents.cell_count(); - } - - let cell_idx = self.stack.current_cell_index() as usize; - - // Second condition is necessary in case we return if the page is locked in the loop below - if contents.is_leaf() || cell_idx > contents.cell_count() { - loop { - if !self.stack.has_parent() { - // All pages of the b-tree have been visited. Return successfully - let c = self.move_to_root()?; - self.count_state = CountState::Finish; - if let Some(c) = c { - io_yield_one!(c); - } - continue 'outer; - } - - // Move to parent - self.stack.pop(); - - mem_page = self.stack.top_ref(); - turso_assert!(mem_page.is_loaded(), "page should be loaded"); - contents = mem_page.get_contents(); - - let cell_idx = self.stack.current_cell_index() as usize; - - if cell_idx <= contents.cell_count() { - break; - } - } - } - - let cell_idx = self.stack.current_cell_index() as usize; - - assert!(cell_idx <= contents.cell_count(),); - assert!(!contents.is_leaf()); - - if cell_idx == contents.cell_count() { - // Move to right child - // should be safe as contents is not a leaf page - let right_most_pointer = contents.rightmost_pointer().unwrap(); - self.stack.advance(); - let (mem_page, c) = self.read_page(right_most_pointer as i64)?; - self.stack.push(mem_page); - if let Some(c) = c { - io_yield_one!(c); - } - } else { - // Move to child left page - let cell = contents.cell_get(cell_idx, self.usable_space())?; - - match cell { - BTreeCell::TableInteriorCell(TableInteriorCell { - left_child_page, - .. - }) - | BTreeCell::IndexInteriorCell(IndexInteriorCell { - left_child_page, - .. - }) => { - self.stack.advance(); - let (mem_page, c) = self.read_page(left_child_page as i64)?; - self.stack.push(mem_page); - if let Some(c) = c { - io_yield_one!(c); - } - } - _ => unreachable!(), - } - } - } - CountState::Finish => { - return Ok(IOResult::Done(self.count)); - } - } - } - } - // Save cursor context, to be restored later pub fn save_context(&mut self, cursor_context: CursorContext) { self.valid_state = CursorValidState::RequireSeek; @@ -5817,6 +5130,18 @@ impl CursorTrait for BTreeCursor { } #[instrument(skip(self), level = Level::DEBUG)] + /// Delete state machine flow: + /// 1. Start -> check if the rowid to be delete is present in the page or not. If not we early return + /// 2. DeterminePostBalancingSeekKey -> determine the key to seek to after balancing. + /// 3. LoadPage -> load the page. + /// 4. FindCell -> find the cell to be deleted in the page. + /// 5. ClearOverflowPages -> Clear the overflow pages if there are any before dropping the cell, then if we are in a leaf page we just drop the cell in place. + /// if we are in interior page, we need to rotate keys in order to replace current cell (InteriorNodeReplacement). + /// 6. InteriorNodeReplacement -> we copy the left subtree leaf node into the deleted interior node's place. + /// 7. Balancing -> perform balancing + /// 8. PostInteriorNodeReplacement -> if an interior node was replaced, we need to advance the cursor once. + /// 9. SeekAfterBalancing -> adjust the cursor to a node that is closer to the deleted value. go to Finish + /// 10. Finish -> Delete operation is done. Return CursorResult(Ok()) fn delete(&mut self) -> Result> { if let Some(mv_cursor) = &self.mv_cursor { return_if_io!(mv_cursor.write().delete()); @@ -6182,6 +5507,9 @@ impl CursorTrait for BTreeCursor { } #[inline(always)] + /// In outer joins, whenever the right-side table has no matching row, the query must still return a row + /// for each left-side row. In order to achieve this, we set the null flag on the right-side table cursor + /// so that it returns NULL for all columns until cleared. fn set_null_flag(&mut self, flag: bool) { self.null_flag = flag; } @@ -6205,16 +5533,30 @@ impl CursorTrait for BTreeCursor { Ok(IOResult::Done(exists)) } + /// Deletes all content from the B-Tree but preserves the root page. + /// + /// Unlike [`btree_destroy`], which frees all pages including the root, + /// this method only clears the tree’s contents. The root page remains + /// allocated and is reset to an empty leaf page. fn clear_btree(&mut self) -> Result>> { self.destroy_btree_contents(true) } + /// Destroys the entire B-Tree, including the root page. + /// + /// All pages belonging to the tree are freed, leaving no trace of the B-Tree. + /// Use this when the structure itself is no longer needed. + /// + /// For cases where the B-Tree should remain allocated but emptied, see [`btree_clear`]. #[instrument(skip(self), level = Level::DEBUG)] fn btree_destroy(&mut self) -> Result>> { self.destroy_btree_contents(false) } #[instrument(skip(self), level = Level::DEBUG)] + /// Count the number of entries in the b-tree + /// + /// Only supposed to be used in the context of a simple Count Select Statement fn count(&mut self) -> Result> { if let Some(_mv_cursor) = &self.mv_cursor { todo!("Implement count for mvcc");