Merge 'core/bree: remove duplicated code in BTreeCursor' from Pere Diaz Bou

Forgot to remove some functions from BTreeCursor moved to CursorTrait.

Reviewed-by: Jussi Saurio <jussi.saurio@gmail.com>

Closes #3751
This commit is contained in:
Pekka Enberg
2025-10-16 17:05:50 +03:00
committed by GitHub
6 changed files with 34 additions and 690 deletions

View File

@@ -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};

View File

@@ -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,

View File

@@ -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,
};

View File

@@ -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};

View File

@@ -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};

View File

@@ -4452,562 +4452,6 @@ impl BTreeCursor {
self.usable_space_cached
}
#[instrument(skip(self), level = Level::DEBUG)]
pub fn rowid(&self) -> Result<IOResult<Option<i64>>> {
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<IOResult<SeekResult>> {
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<IOResult<Option<Ref<'_, ImmutableRecord>>>> {
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<IOResult<()>> {
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<IOResult<()>> {
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<u8> = 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<IOResult<bool>> {
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 trees contents. The root page remains
/// allocated and is reset to an empty leaf page.
pub fn clear_btree(&mut self) -> Result<IOResult<Option<usize>>> {
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<IOResult<Option<usize>>> {
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<IOResult<usize>> {
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<IOResult<()>> {
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 trees contents. The root page remains
/// allocated and is reset to an empty leaf page.
fn clear_btree(&mut self) -> Result<IOResult<Option<usize>>> {
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<IOResult<Option<usize>>> {
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<IOResult<usize>> {
if let Some(_mv_cursor) = &self.mv_cursor {
todo!("Implement count for mvcc");