Merge 'core/mvcc: Move commit_txn() to generic state machinery ' from Pere Diaz Bou

Unfortunately it seems we are never reaching the point to remove state
machines, so might as well make it easier to make.
There are two points that must be highlighted:
1. There is a `StateTransition` trait implemented like:
```rust
pub trait StateTransition {
    type State;
    type Context;

    fn transition<'a>(&mut self, context: &Self::Context) ->
Result<TransitionResult>;
    fn finalize<'a>(&mut self, context: &Self::Context) -> Result<()>;
    fn is_finalized(&self) -> bool;
}
```
where there exists `transition` which tries to move state forward, and
`finalize` which marks the state machine as "finalized" so that **no
other call to finalize will forward the state and it will panic instead.
2. Before, we would store the state of a state machine inside the
callee's struct, but I'm proposing we do something different where the
callee will return the state machine and the caller will be responsible
of advancing it. This way we don't need to track many reset operations
in case of failures or rollbacks, and instead we could simply drop a
state machine and all other nested state machines will drop in a
cascade.

Closes #2384
This commit is contained in:
Pekka Enberg
2025-08-01 19:28:16 +03:00
committed by GitHub
10 changed files with 637 additions and 360 deletions

View File

@@ -63,6 +63,12 @@ pub enum LimboError {
Busy,
#[error("Conflict: {0}")]
Conflict(String),
#[error("Transaction terminated")]
TxTerminated,
#[error("Write-write conflict")]
WriteWriteConflict,
#[error("No such transaction ID: {0}")]
NoSuchTransactionID(String),
}
#[macro_export]

View File

@@ -18,6 +18,7 @@ pub mod result;
mod schema;
#[cfg(feature = "series")]
mod series;
mod state_machine;
mod storage;
#[allow(dead_code)]
#[cfg(feature = "time")]

View File

@@ -1,6 +1,7 @@
use crate::mvcc::clock::LogicalClock;
use crate::mvcc::database::{MvStore, Result, Row, RowID};
use crate::mvcc::database::{MvStore, Row, RowID};
use crate::Pager;
use crate::Result;
use std::fmt::Debug;
use std::rc::Rc;
use std::sync::Arc;

View File

@@ -1,21 +1,24 @@
use crate::mvcc::clock::LogicalClock;
use crate::mvcc::errors::DatabaseError;
use crate::mvcc::persistent_storage::Storage;
use crate::state_machine::StateMachine;
use crate::state_machine::StateTransition;
use crate::state_machine::TransitionResult;
use crate::storage::btree::BTreeCursor;
use crate::storage::btree::BTreeKey;
use crate::types::IOResult;
use crate::types::ImmutableRecord;
use crate::LimboError;
use crate::Result;
use crate::{Connection, Pager};
use crossbeam_skiplist::{SkipMap, SkipSet};
use parking_lot::RwLock;
use std::collections::HashSet;
use std::fmt::Debug;
use std::marker::PhantomData;
use std::rc::Rc;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::Arc;
pub type Result<T> = std::result::Result<T, DatabaseError>;
#[cfg(test)]
pub mod tests;
@@ -235,6 +238,444 @@ impl AtomicTransactionState {
}
}
#[derive(Debug)]
pub enum CommitState {
Initial,
BeginPagerTxn { end_ts: u64 },
WriteRow { end_ts: u64, write_set_index: usize },
WriteRowStateMachine { end_ts: u64, write_set_index: usize },
CommitPagerTxn { end_ts: u64 },
Commit { end_ts: u64 },
}
#[derive(Debug)]
pub enum WriteRowState {
Initial,
CreateCursor,
Seek,
Insert,
}
pub struct CommitStateMachine<Clock: LogicalClock> {
state: CommitState,
is_finalized: bool,
pager: Rc<Pager>,
tx_id: TxID,
connection: Arc<Connection>,
write_set: Vec<RowID>,
write_row_state_machine: Option<StateMachine<WriteRowStateMachine>>,
_phantom: PhantomData<Clock>,
}
pub struct WriteRowStateMachine {
state: WriteRowState,
is_finalized: bool,
pager: Rc<Pager>,
row: Row,
record: Option<ImmutableRecord>,
cursor: Option<BTreeCursor>,
}
impl<Clock: LogicalClock> CommitStateMachine<Clock> {
fn new(state: CommitState, pager: Rc<Pager>, tx_id: TxID, connection: Arc<Connection>) -> Self {
Self {
state,
is_finalized: false,
pager,
tx_id,
connection,
write_set: Vec::new(),
write_row_state_machine: None,
_phantom: PhantomData,
}
}
}
impl WriteRowStateMachine {
fn new(pager: Rc<Pager>, row: Row) -> Self {
Self {
state: WriteRowState::Initial,
is_finalized: false,
pager,
row,
record: None,
cursor: None,
}
}
}
impl<Clock: LogicalClock> StateTransition for CommitStateMachine<Clock> {
type State = CommitStateMachine<Clock>;
type Context = MvStore<Clock>;
type SMResult = ();
#[tracing::instrument(fields(state = ?self.state), skip(self, mvcc_store))]
fn step(&mut self, mvcc_store: &Self::Context) -> Result<TransitionResult<Self::SMResult>> {
match self.state {
CommitState::Initial => {
let end_ts = mvcc_store.get_timestamp();
// NOTICE: the first shadowed tx keeps the entry alive in the map
// for the duration of this whole function, which is important for correctness!
let tx = mvcc_store
.txs
.get(&self.tx_id)
.ok_or(LimboError::TxTerminated)?;
let tx = tx.value().write();
match tx.state.load() {
TransactionState::Terminated => {
return Err(LimboError::TxTerminated);
}
_ => {
assert_eq!(tx.state, TransactionState::Active);
}
}
tx.state.store(TransactionState::Preparing);
tracing::trace!("prepare_tx(tx_id={})", self.tx_id);
/* TODO: The code we have here is sufficient for snapshot isolation.
** In order to implement serializability, we need the following steps:
**
** 1. Validate if all read versions are still visible by inspecting the read_set
** 2. Validate if there are no phantoms by walking the scans from scan_set (which we don't even have yet)
** - a phantom is a version that became visible in the middle of our transaction,
** but wasn't taken into account during one of the scans from the scan_set
** 3. Wait for commit dependencies, which we don't even track yet...
** Excerpt from what's a commit dependency and how it's tracked in the original paper:
** """
A transaction T1 has a commit dependency on another transaction
T2, if T1 is allowed to commit only if T2 commits. If T2 aborts,
T1 must also abort, so cascading aborts are possible. T1 acquires a
commit dependency either by speculatively reading or speculatively ignoring a version,
instead of waiting for T2 to commit.
We implement commit dependencies by a register-and-report
approach: T1 registers its dependency with T2 and T2 informs T1
when it has committed or aborted. Each transaction T contains a
counter, CommitDepCounter, that counts how many unresolved
commit dependencies it still has. A transaction cannot commit
until this counter is zero. In addition, T has a Boolean variable
AbortNow that other transactions can set to tell T to abort. Each
transaction T also has a set, CommitDepSet, that stores transaction IDs
of the transactions that depend on T.
To take a commit dependency on a transaction T2, T1 increments
its CommitDepCounter and adds its transaction ID to T2s CommitDepSet.
When T2 has committed, it locates each transaction in
its CommitDepSet and decrements their CommitDepCounter. If
T2 aborted, it tells the dependent transactions to also abort by
setting their AbortNow flags. If a dependent transaction is not
found, this means that it has already aborted.
Note that a transaction with commit dependencies may not have to
wait at all - the dependencies may have been resolved before it is
ready to commit. Commit dependencies consolidate all waits into
a single wait and postpone the wait to just before commit.
Some transactions may have to wait before commit.
Waiting raises a concern of deadlocks.
However, deadlocks cannot occur because an older transaction never
waits on a younger transaction. In
a wait-for graph the direction of edges would always be from a
younger transaction (higher end timestamp) to an older transaction
(lower end timestamp) so cycles are impossible.
"""
** If you're wondering when a speculative read happens, here you go:
** Case 1: speculative read of TB:
"""
If transaction TB is in the Preparing state, it has acquired an end
timestamp TS which will be Vs begin timestamp if TB commits.
A safe approach in this situation would be to have transaction T
wait until transaction TB commits. However, we want to avoid all
blocking during normal processing so instead we continue with
the visibility test and, if the test returns true, allow T to
speculatively read V. Transaction T acquires a commit dependency on
TB, restricting the serialization order of the two transactions. That
is, T is allowed to commit only if TB commits.
"""
** Case 2: speculative ignore of TE:
"""
If TEs state is Preparing, it has an end timestamp TS that will become
the end timestamp of V if TE does commit. If TS is greater than the read
time RT, it is obvious that V will be visible if TE commits. If TE
aborts, V will still be visible, because any transaction that updates
V after TE has aborted will obtain an end timestamp greater than
TS. If TS is less than RT, we have a more complicated situation:
if TE commits, V will not be visible to T but if TE aborts, it will
be visible. We could handle this by forcing T to wait until TE
commits or aborts but we want to avoid all blocking during normal processing.
Instead we allow T to speculatively ignore V and
proceed with its processing. Transaction T acquires a commit
dependency (see Section 2.7) on TE, that is, T is allowed to commit
only if TE commits.
"""
*/
tx.state.store(TransactionState::Committed(end_ts));
tracing::trace!("commit_tx(tx_id={})", self.tx_id);
self.write_set
.extend(tx.write_set.iter().map(|v| *v.value()));
self.state = CommitState::BeginPagerTxn { end_ts };
Ok(TransitionResult::Continue)
}
CommitState::BeginPagerTxn { end_ts } => {
// FIXME: how do we deal with multiple concurrent writes?
// WAL requires a txn to be written sequentially. Either we:
// 1. Wait for currently writer to finish before second txn starts.
// 2. Choose a txn to write depending on some heuristics like amount of frames will be written.
// 3. ..
//
loop {
match self.pager.begin_write_tx() {
Ok(crate::types::IOResult::Done(result)) => {
if let crate::result::LimboResult::Busy = result {
return Err(LimboError::InternalError(
"Pager write transaction busy".to_string(),
));
}
break;
}
Ok(crate::types::IOResult::IO) => {
// FIXME: this is a hack to make the pager run the IO loop
self.pager.io.run_once().unwrap();
continue;
}
Err(e) => {
return Err(LimboError::InternalError(e.to_string()));
}
}
}
self.state = CommitState::WriteRow {
end_ts,
write_set_index: 0,
};
return Ok(TransitionResult::Continue);
}
CommitState::WriteRow {
end_ts,
write_set_index,
} => {
if write_set_index == self.write_set.len() {
self.state = CommitState::CommitPagerTxn { end_ts };
return Ok(TransitionResult::Continue);
}
let id = &self.write_set[write_set_index];
if let Some(row_versions) = mvcc_store.rows.get(id) {
let row_versions = row_versions.value().read();
// Find rows that were written by this transaction
for row_version in row_versions.iter() {
if let TxTimestampOrID::TxID(row_tx_id) = row_version.begin {
if row_tx_id == self.tx_id {
let state_machine = mvcc_store
.write_row_to_pager(self.pager.clone(), &row_version.row)?;
self.write_row_state_machine = Some(state_machine);
self.state = CommitState::WriteRowStateMachine {
end_ts,
write_set_index,
};
break;
}
}
if let Some(TxTimestampOrID::Timestamp(row_tx_id)) = row_version.end {
if row_tx_id == self.tx_id {
let state_machine = mvcc_store
.write_row_to_pager(self.pager.clone(), &row_version.row)?;
self.write_row_state_machine = Some(state_machine);
self.state = CommitState::WriteRowStateMachine {
end_ts,
write_set_index,
};
break;
}
}
}
}
Ok(TransitionResult::Continue)
}
CommitState::WriteRowStateMachine {
end_ts,
write_set_index,
} => {
let write_row_state_machine = self.write_row_state_machine.as_mut().unwrap();
match write_row_state_machine.step(&())? {
TransitionResult::Io => return Ok(TransitionResult::Io),
TransitionResult::Continue => {
return Ok(TransitionResult::Continue);
}
TransitionResult::Done(_) => {
self.state = CommitState::WriteRow {
end_ts,
write_set_index: write_set_index + 1,
};
return Ok(TransitionResult::Continue);
}
}
}
CommitState::CommitPagerTxn { end_ts } => {
// Write committed data to pager for persistence
// Flush dirty pages to WAL - this is critical for data persistence
// Similar to what step_end_write_txn does for legacy transactions
loop {
let result = self
.pager
.end_tx(
false, // rollback = false since we're committing
false, // schema_did_change = false for now (could be improved)
&self.connection,
self.connection.wal_checkpoint_disabled.get(),
)
.map_err(|e| LimboError::InternalError(e.to_string()))
.unwrap();
if let crate::types::IOResult::Done(_) = result {
break;
}
}
self.state = CommitState::Commit { end_ts };
Ok(TransitionResult::Continue)
}
CommitState::Commit { end_ts } => {
let mut log_record = LogRecord::new(end_ts);
for id in &self.write_set {
if let Some(row_versions) = mvcc_store.rows.get(id) {
let mut row_versions = row_versions.value().write();
for row_version in row_versions.iter_mut() {
if let TxTimestampOrID::TxID(id) = row_version.begin {
if id == self.tx_id {
// New version is valid STARTING FROM committing transaction's end timestamp
// See diagram on page 299: https://www.cs.cmu.edu/~15721-f24/papers/Hekaton.pdf
row_version.begin = TxTimestampOrID::Timestamp(end_ts);
mvcc_store.insert_version_raw(
&mut log_record.row_versions,
row_version.clone(),
); // FIXME: optimize cloning out
}
}
if let Some(TxTimestampOrID::TxID(id)) = row_version.end {
if id == self.tx_id {
// Old version is valid UNTIL committing transaction's end timestamp
// See diagram on page 299: https://www.cs.cmu.edu/~15721-f24/papers/Hekaton.pdf
row_version.end = Some(TxTimestampOrID::Timestamp(end_ts));
mvcc_store.insert_version_raw(
&mut log_record.row_versions,
row_version.clone(),
); // FIXME: optimize cloning out
}
}
}
}
}
tracing::trace!("updated(tx_id={})", self.tx_id);
// We have now updated all the versions with a reference to the
// transaction ID to a timestamp and can, therefore, remove the
// transaction. Please note that when we move to lockless, the
// invariant doesn't necessarily hold anymore because another thread
// might have speculatively read a version that we want to remove.
// But that's a problem for another day.
// FIXME: it actually just become a problem for today!!!
// TODO: test that reproduces this failure, and then a fix
mvcc_store.txs.remove(&self.tx_id);
if !log_record.row_versions.is_empty() {
mvcc_store.storage.log_tx(log_record)?;
}
tracing::trace!("logged(tx_id={})", self.tx_id);
self.finalize(mvcc_store)?;
Ok(TransitionResult::Done(()))
}
}
}
fn finalize(&mut self, _context: &Self::Context) -> Result<()> {
self.is_finalized = true;
Ok(())
}
fn is_finalized(&self) -> bool {
self.is_finalized
}
}
impl StateTransition for WriteRowStateMachine {
type State = WriteRowStateMachine;
type Context = ();
type SMResult = ();
#[tracing::instrument(fields(state = ?self.state), skip(self, _context))]
fn step(&mut self, _context: &Self::Context) -> Result<TransitionResult<Self::SMResult>> {
use crate::storage::btree::BTreeCursor;
use crate::types::{IOResult, SeekKey, SeekOp};
match self.state {
WriteRowState::Initial => {
// Create the record and key
let mut record = ImmutableRecord::new(self.row.data.len());
record.start_serialization(&self.row.data);
self.record = Some(record);
self.state = WriteRowState::CreateCursor;
Ok(TransitionResult::Continue)
}
WriteRowState::CreateCursor => {
// Create the cursor
let root_page = self.row.id.table_id as usize;
let num_columns = self.row.column_count;
let cursor = BTreeCursor::new_table(
None, // Write directly to B-tree
self.pager.clone(),
root_page,
num_columns,
);
self.cursor = Some(cursor);
self.state = WriteRowState::Seek;
Ok(TransitionResult::Continue)
}
WriteRowState::Seek => {
// Position the cursor by seeking to the row position
let seek_key = SeekKey::TableRowId(self.row.id.row_id);
let cursor = self.cursor.as_mut().unwrap();
match cursor.seek(seek_key, SeekOp::GE { eq_only: true })? {
IOResult::Done(_) => {
self.state = WriteRowState::Insert;
Ok(TransitionResult::Continue)
}
IOResult::IO => {
return Ok(TransitionResult::Io);
}
}
}
WriteRowState::Insert => {
// Insert the record into the B-tree
let cursor = self.cursor.as_mut().unwrap();
let key = BTreeKey::new_table_rowid(self.row.id.row_id, self.record.as_ref());
match cursor
.insert(&key, true)
.map_err(|e| LimboError::InternalError(e.to_string()))?
{
IOResult::Done(()) => {
tracing::trace!(
"write_row_to_pager(table_id={}, row_id={})",
self.row.id.table_id,
self.row.id.row_id
);
self.finalize(&())?;
Ok(TransitionResult::Done(()))
}
IOResult::IO => {
return Ok(TransitionResult::Io);
}
}
}
}
}
fn finalize(&mut self, _context: &Self::Context) -> Result<()> {
self.is_finalized = true;
Ok(())
}
fn is_finalized(&self) -> bool {
self.is_finalized
}
}
/// A multi-version concurrency control database.
#[derive(Debug)]
pub struct MvStore<Clock: LogicalClock> {
@@ -280,7 +721,7 @@ impl<Clock: LogicalClock> MvStore<Clock> {
let tx = self
.txs
.get(&tx_id)
.ok_or(DatabaseError::NoSuchTransactionID(tx_id))?;
.ok_or(LimboError::NoSuchTransactionID(tx_id.to_string()))?;
let mut tx = tx.value().write();
assert_eq!(tx.state, TransactionState::Active);
let id = row.id;
@@ -353,7 +794,7 @@ impl<Clock: LogicalClock> MvStore<Clock> {
let tx = self
.txs
.get(&tx_id)
.ok_or(DatabaseError::NoSuchTransactionID(tx_id))?;
.ok_or(LimboError::NoSuchTransactionID(tx_id.to_string()))?;
let tx = tx.value().read();
assert_eq!(tx.state, TransactionState::Active);
// A transaction cannot delete a version that it cannot see,
@@ -366,7 +807,7 @@ impl<Clock: LogicalClock> MvStore<Clock> {
drop(row_versions_opt);
drop(tx);
self.rollback_tx(tx_id, pager);
return Err(DatabaseError::WriteWriteConflict);
return Err(LimboError::WriteWriteConflict);
}
rv.end = Some(TxTimestampOrID::TxID(tx.tx_id));
@@ -376,7 +817,7 @@ impl<Clock: LogicalClock> MvStore<Clock> {
let tx = self
.txs
.get(&tx_id)
.ok_or(DatabaseError::NoSuchTransactionID(tx_id))?;
.ok_or(LimboError::NoSuchTransactionID(tx_id.to_string()))?;
let mut tx = tx.value().write();
tx.insert_to_write_set(id);
return Ok(true);
@@ -509,212 +950,13 @@ impl<Clock: LogicalClock> MvStore<Clock> {
tx_id: TxID,
pager: Rc<Pager>,
connection: &Arc<Connection>,
) -> Result<()> {
let end_ts = self.get_timestamp();
// NOTICE: the first shadowed tx keeps the entry alive in the map
// for the duration of this whole function, which is important for correctness!
let tx = self.txs.get(&tx_id).ok_or(DatabaseError::TxTerminated)?;
let tx = tx.value().write();
match tx.state.load() {
TransactionState::Terminated => return Err(DatabaseError::TxTerminated),
_ => {
assert_eq!(tx.state, TransactionState::Active);
}
}
tx.state.store(TransactionState::Preparing);
tracing::trace!("prepare_tx(tx_id={})", tx_id);
/* TODO: The code we have here is sufficient for snapshot isolation.
** In order to implement serializability, we need the following steps:
**
** 1. Validate if all read versions are still visible by inspecting the read_set
** 2. Validate if there are no phantoms by walking the scans from scan_set (which we don't even have yet)
** - a phantom is a version that became visible in the middle of our transaction,
** but wasn't taken into account during one of the scans from the scan_set
** 3. Wait for commit dependencies, which we don't even track yet...
** Excerpt from what's a commit dependency and how it's tracked in the original paper:
** """
A transaction T1 has a commit dependency on another transaction
T2, if T1 is allowed to commit only if T2 commits. If T2 aborts,
T1 must also abort, so cascading aborts are possible. T1 acquires a
commit dependency either by speculatively reading or speculatively ignoring a version,
instead of waiting for T2 to commit.
We implement commit dependencies by a register-and-report
approach: T1 registers its dependency with T2 and T2 informs T1
when it has committed or aborted. Each transaction T contains a
counter, CommitDepCounter, that counts how many unresolved
commit dependencies it still has. A transaction cannot commit
until this counter is zero. In addition, T has a Boolean variable
AbortNow that other transactions can set to tell T to abort. Each
transaction T also has a set, CommitDepSet, that stores transaction IDs
of the transactions that depend on T.
To take a commit dependency on a transaction T2, T1 increments
its CommitDepCounter and adds its transaction ID to T2s CommitDepSet.
When T2 has committed, it locates each transaction in
its CommitDepSet and decrements their CommitDepCounter. If
T2 aborted, it tells the dependent transactions to also abort by
setting their AbortNow flags. If a dependent transaction is not
found, this means that it has already aborted.
Note that a transaction with commit dependencies may not have to
wait at all - the dependencies may have been resolved before it is
ready to commit. Commit dependencies consolidate all waits into
a single wait and postpone the wait to just before commit.
Some transactions may have to wait before commit.
Waiting raises a concern of deadlocks.
However, deadlocks cannot occur because an older transaction never
waits on a younger transaction. In
a wait-for graph the direction of edges would always be from a
younger transaction (higher end timestamp) to an older transaction
(lower end timestamp) so cycles are impossible.
"""
** If you're wondering when a speculative read happens, here you go:
** Case 1: speculative read of TB:
"""
If transaction TB is in the Preparing state, it has acquired an end
timestamp TS which will be Vs begin timestamp if TB commits.
A safe approach in this situation would be to have transaction T
wait until transaction TB commits. However, we want to avoid all
blocking during normal processing so instead we continue with
the visibility test and, if the test returns true, allow T to
speculatively read V. Transaction T acquires a commit dependency on
TB, restricting the serialization order of the two transactions. That
is, T is allowed to commit only if TB commits.
"""
** Case 2: speculative ignore of TE:
"""
If TEs state is Preparing, it has an end timestamp TS that will become
the end timestamp of V if TE does commit. If TS is greater than the read
time RT, it is obvious that V will be visible if TE commits. If TE
aborts, V will still be visible, because any transaction that updates
V after TE has aborted will obtain an end timestamp greater than
TS. If TS is less than RT, we have a more complicated situation:
if TE commits, V will not be visible to T but if TE aborts, it will
be visible. We could handle this by forcing T to wait until TE
commits or aborts but we want to avoid all blocking during normal processing.
Instead we allow T to speculatively ignore V and
proceed with its processing. Transaction T acquires a commit
dependency (see Section 2.7) on TE, that is, T is allowed to commit
only if TE commits.
"""
*/
tx.state.store(TransactionState::Committed(end_ts));
tracing::trace!("commit_tx(tx_id={})", tx_id);
let write_set: Vec<RowID> = tx.write_set.iter().map(|v| *v.value()).collect();
drop(tx);
// Postprocessing: inserting row versions and logging the transaction to persistent storage.
// FIXME: how do we deal with multiple concurrent writes?
// WAL requires a txn to be written sequentially. Either we:
// 1. Wait for currently writer to finish before second txn starts.
// 2. Choose a txn to write depending on some heuristics like amount of frames will be written.
// 3. ..
//
loop {
match pager.begin_write_tx() {
Ok(crate::types::IOResult::Done(result)) => {
if let crate::result::LimboResult::Busy = result {
return Err(DatabaseError::Io(
"Pager write transaction busy".to_string(),
));
}
break;
}
Ok(crate::types::IOResult::IO) => {
// FIXME: this is a hack to make the pager run the IO loop
pager.io.run_once().unwrap();
continue;
}
Err(e) => {
return Err(DatabaseError::Io(e.to_string()));
}
}
}
// 1. Write rows to btree for persistence
for id in &write_set {
if let Some(row_versions) = self.rows.get(id) {
let row_versions = row_versions.value().read();
// Find rows that were written by this transaction
for row_version in row_versions.iter() {
if let TxTimestampOrID::TxID(row_tx_id) = row_version.begin {
if row_tx_id == tx_id {
self.write_row_to_pager(pager.clone(), &row_version.row)?;
break;
}
}
if let Some(TxTimestampOrID::Timestamp(row_tx_id)) = row_version.end {
if row_tx_id == tx_id {
self.write_row_to_pager(pager.clone(), &row_version.row)?;
break;
}
}
}
}
}
// Write committed data to pager for persistence
// Flush dirty pages to WAL - this is critical for data persistence
// Similar to what step_end_write_txn does for legacy transactions
loop {
let result = pager
.end_tx(
false, // rollback = false since we're committing
false, // schema_did_change = false for now (could be improved)
connection,
connection.wal_checkpoint_disabled.get(),
)
.map_err(|e| DatabaseError::Io(e.to_string()))
.unwrap();
if let crate::types::IOResult::Done(_) = result {
break;
}
}
// 2. Commit rows to log
let mut log_record = LogRecord::new(end_ts);
for ref id in write_set {
if let Some(row_versions) = self.rows.get(id) {
let mut row_versions = row_versions.value().write();
for row_version in row_versions.iter_mut() {
if let TxTimestampOrID::TxID(id) = row_version.begin {
if id == tx_id {
// New version is valid STARTING FROM committing transaction's end timestamp
// See diagram on page 299: https://www.cs.cmu.edu/~15721-f24/papers/Hekaton.pdf
row_version.begin = TxTimestampOrID::Timestamp(end_ts);
self.insert_version_raw(
&mut log_record.row_versions,
row_version.clone(),
); // FIXME: optimize cloning out
}
}
if let Some(TxTimestampOrID::TxID(id)) = row_version.end {
if id == tx_id {
// Old version is valid UNTIL committing transaction's end timestamp
// See diagram on page 299: https://www.cs.cmu.edu/~15721-f24/papers/Hekaton.pdf
row_version.end = Some(TxTimestampOrID::Timestamp(end_ts));
self.insert_version_raw(
&mut log_record.row_versions,
row_version.clone(),
); // FIXME: optimize cloning out
}
}
}
}
}
tracing::trace!("updated(tx_id={})", tx_id);
// We have now updated all the versions with a reference to the
// transaction ID to a timestamp and can, therefore, remove the
// transaction. Please note that when we move to lockless, the
// invariant doesn't necessarily hold anymore because another thread
// might have speculatively read a version that we want to remove.
// But that's a problem for another day.
// FIXME: it actually just become a problem for today!!!
// TODO: test that reproduces this failure, and then a fix
self.txs.remove(&tx_id);
if !log_record.row_versions.is_empty() {
self.storage.log_tx(log_record)?;
}
tracing::trace!("logged(tx_id={})", tx_id);
Ok(())
) -> Result<StateMachine<CommitStateMachine<Clock>>> {
let state_machine: StateMachine<CommitStateMachine<Clock>> = StateMachine::<
CommitStateMachine<Clock>,
>::new(
CommitStateMachine::new(CommitState::Initial, pager, tx_id, connection.clone()),
);
Ok(state_machine)
}
/// Rolls back a transaction with the specified ID.
@@ -851,7 +1093,7 @@ impl<Clock: LogicalClock> MvStore<Clock> {
/// Inserts a new row version into the internal data structure for versions,
/// while making sure that the row version is inserted in the correct order.
fn insert_version_raw(&self, versions: &mut Vec<RowVersion>, row_version: RowVersion) {
pub fn insert_version_raw(&self, versions: &mut Vec<RowVersion>, row_version: RowVersion) {
// NOTICE: this is an insert a'la insertion sort, with pessimistic linear complexity.
// However, we expect the number of versions to be nearly sorted, so we deem it worthy
// to search linearly for the insertion point instead of paying the price of using
@@ -874,64 +1116,18 @@ impl<Clock: LogicalClock> MvStore<Clock> {
versions.insert(position, row_version);
}
fn write_row_to_pager(&self, pager: Rc<Pager>, row: &Row) -> Result<()> {
use crate::storage::btree::BTreeCursor;
use crate::types::{IOResult, SeekKey, SeekOp};
pub fn write_row_to_pager(
&self,
pager: Rc<Pager>,
row: &Row,
) -> Result<StateMachine<WriteRowStateMachine>> {
let state_machine: StateMachine<WriteRowStateMachine> =
StateMachine::<WriteRowStateMachine>::new(WriteRowStateMachine::new(
pager,
row.clone(),
));
// The row.data is already a properly serialized SQLite record payload
// Create an ImmutableRecord and copy the data
let mut record = ImmutableRecord::new(row.data.len());
record.start_serialization(&row.data);
// Create a BTreeKey for the row
let key = BTreeKey::new_table_rowid(row.id.row_id, Some(&record));
// Get the column count from the row
let root_page = row.id.table_id as usize;
let num_columns = row.column_count;
let mut cursor = BTreeCursor::new_table(
None, // Write directly to B-tree
pager.clone(),
root_page,
num_columns,
);
// Position the cursor first by seeking to the row position
let seek_key = SeekKey::TableRowId(row.id.row_id);
match cursor
.seek(seek_key, SeekOp::GE { eq_only: true })
.map_err(|e| DatabaseError::Io(e.to_string()))?
{
IOResult::Done(_) => {}
IOResult::IO => {
panic!("IOResult::IO not supported in write_row_to_pager seek");
}
}
// Insert the record into the B-tree
loop {
match cursor
.insert(&key, true)
.map_err(|e| DatabaseError::Io(e.to_string()))
{
Ok(IOResult::Done(())) => break,
Ok(IOResult::IO) => {
pager.io.run_once().unwrap();
continue;
}
Err(e) => {
return Err(DatabaseError::Io(e.to_string()));
}
}
}
tracing::trace!(
"write_row_to_pager(table_id={}, row_id={})",
row.id.table_id,
row.id.row_id
);
Ok(())
Ok(state_machine)
}
/// Try to scan for row ids in the table.
@@ -971,7 +1167,7 @@ impl<Clock: LogicalClock> MvStore<Clock> {
loop {
match cursor
.rewind()
.map_err(|e| DatabaseError::Io(e.to_string()))?
.map_err(|e| LimboError::InternalError(e.to_string()))?
{
IOResult::Done(()) => break,
IOResult::IO => {
@@ -983,7 +1179,7 @@ impl<Clock: LogicalClock> MvStore<Clock> {
loop {
let rowid_result = cursor
.rowid()
.map_err(|e| DatabaseError::Io(e.to_string()))?;
.map_err(|e| LimboError::InternalError(e.to_string()))?;
let row_id = match rowid_result {
IOResult::Done(Some(row_id)) => row_id,
IOResult::Done(None) => break,
@@ -994,7 +1190,7 @@ impl<Clock: LogicalClock> MvStore<Clock> {
};
match cursor
.record()
.map_err(|e| DatabaseError::Io(e.to_string()))?
.map_err(|e| LimboError::InternalError(e.to_string()))?
{
IOResult::Done(Some(record)) => {
let id = RowID { table_id, row_id };
@@ -1016,7 +1212,7 @@ impl<Clock: LogicalClock> MvStore<Clock> {
// Move to next record
match cursor
.next()
.map_err(|e| DatabaseError::Io(e.to_string()))?
.map_err(|e| LimboError::InternalError(e.to_string()))?
{
IOResult::Done(has_next) => {
if !has_next {

View File

@@ -64,9 +64,7 @@ fn test_insert_read() {
.unwrap()
.unwrap();
assert_eq!(tx1_row, row);
db.mvcc_store
.commit_tx(tx1, db.conn.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &db.conn, tx1).unwrap();
let tx2 = db.mvcc_store.begin_tx(db.conn.pager.borrow().clone());
let row = db
@@ -137,9 +135,7 @@ fn test_delete() {
)
.unwrap();
assert!(row.is_none());
db.mvcc_store
.commit_tx(tx1, db.conn.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &db.conn, tx1).unwrap();
let tx2 = db.mvcc_store.begin_tx(db.conn.pager.borrow().clone());
let row = db
@@ -206,9 +202,7 @@ fn test_commit() {
.unwrap()
.unwrap();
assert_eq!(tx1_updated_row, row);
db.mvcc_store
.commit_tx(tx1, db.conn.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &db.conn, tx1).unwrap();
let tx2 = db.mvcc_store.begin_tx(db.conn.pager.borrow().clone());
let row = db
@@ -222,9 +216,7 @@ fn test_commit() {
)
.unwrap()
.unwrap();
db.mvcc_store
.commit_tx(tx2, db.conn.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &db.conn, tx2).unwrap();
assert_eq!(tx1_updated_row, row);
db.mvcc_store.drop_unused_row_versions();
}
@@ -356,9 +348,7 @@ fn test_dirty_read_deleted() {
let tx1 = db.mvcc_store.begin_tx(db.conn.pager.borrow().clone());
let tx1_row = generate_simple_string_row(1, 1, "Hello");
db.mvcc_store.insert(tx1, tx1_row.clone()).unwrap();
db.mvcc_store
.commit_tx(tx1, db.conn.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &db.conn, tx1).unwrap();
// T2 deletes row with ID 1, but does not commit.
let conn2 = db._db.connect().unwrap();
@@ -412,9 +402,7 @@ fn test_fuzzy_read() {
.unwrap()
.unwrap();
assert_eq!(tx1_row, row);
db.mvcc_store
.commit_tx(tx1, db.conn.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &db.conn, tx1).unwrap();
// T2 reads the row with ID 1 within an active transaction.
let conn2 = db._db.connect().unwrap();
@@ -439,9 +427,7 @@ fn test_fuzzy_read() {
db.mvcc_store
.update(tx3, tx3_row, conn3.pager.borrow().clone())
.unwrap();
db.mvcc_store
.commit_tx(tx3, conn3.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &conn3, tx3).unwrap();
// T2 still reads the same version of the row as before.
let row = db
@@ -463,7 +449,7 @@ fn test_fuzzy_read() {
let update_result = db
.mvcc_store
.update(tx2, tx2_newrow, conn2.pager.borrow().clone());
assert_eq!(Err(DatabaseError::WriteWriteConflict), update_result);
assert!(matches!(update_result, Err(LimboError::WriteWriteConflict)));
}
#[test]
@@ -486,9 +472,7 @@ fn test_lost_update() {
.unwrap()
.unwrap();
assert_eq!(tx1_row, row);
db.mvcc_store
.commit_tx(tx1, db.conn.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &db.conn, tx1).unwrap();
// T2 attempts to update row ID 1 within an active transaction.
let conn2 = db._db.connect().unwrap();
@@ -503,20 +487,17 @@ fn test_lost_update() {
let conn3 = db._db.connect().unwrap();
let tx3 = db.mvcc_store.begin_tx(conn3.pager.borrow().clone());
let tx3_row = generate_simple_string_row(1, 1, "Hello, world!");
assert_eq!(
Err(DatabaseError::WriteWriteConflict),
assert!(matches!(
db.mvcc_store
.update(tx3, tx3_row, conn3.pager.borrow().clone())
);
.update(tx3, tx3_row, conn3.pager.borrow().clone(),),
Err(LimboError::WriteWriteConflict)
));
db.mvcc_store
.commit_tx(tx2, conn2.pager.borrow().clone(), &db.conn)
.unwrap();
assert_eq!(
Err(DatabaseError::TxTerminated),
db.mvcc_store
.commit_tx(tx3, conn3.pager.borrow().clone(), &db.conn)
);
commit_tx(db.mvcc_store.clone(), &conn2, tx2).unwrap();
assert!(matches!(
commit_tx(db.mvcc_store.clone(), &conn3, tx3),
Err(LimboError::TxTerminated)
));
let conn4 = db._db.connect().unwrap();
let tx4 = db.mvcc_store.begin_tx(conn4.pager.borrow().clone());
@@ -544,9 +525,7 @@ fn test_committed_visibility() {
let tx1 = db.mvcc_store.begin_tx(db.conn.pager.borrow().clone());
let tx1_row = generate_simple_string_row(1, 1, "10");
db.mvcc_store.insert(tx1, tx1_row.clone()).unwrap();
db.mvcc_store
.commit_tx(tx1, db.conn.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &db.conn, tx1).unwrap();
// but I like more money, so let me try adding $10 more
let conn2 = db._db.connect().unwrap();
@@ -612,9 +591,7 @@ fn test_future_row() {
assert_eq!(row, None);
// lets commit the transaction and check if tx1 can see it
db.mvcc_store
.commit_tx(tx2, conn2.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &conn2, tx2).unwrap();
let row = db
.mvcc_store
.read(
@@ -658,9 +635,7 @@ fn setup_test_db() -> (MvccTestDb, u64) {
db.mvcc_store.insert(tx_id, row).unwrap();
}
db.mvcc_store
.commit_tx(tx_id, db.conn.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &db.conn, tx_id).unwrap();
let tx_id = db.mvcc_store.begin_tx(db.conn.pager.borrow().clone());
(db, tx_id)
@@ -679,14 +654,47 @@ fn setup_lazy_db(initial_keys: &[i64]) -> (MvccTestDb, u64) {
db.mvcc_store.insert(tx_id, row).unwrap();
}
db.mvcc_store
.commit_tx(tx_id, db.conn.pager.borrow().clone(), &db.conn)
.unwrap();
commit_tx(db.mvcc_store.clone(), &db.conn, tx_id).unwrap();
let tx_id = db.mvcc_store.begin_tx(db.conn.pager.borrow().clone());
(db, tx_id)
}
pub(crate) fn commit_tx(
mv_store: Arc<MvStore<LocalClock>>,
conn: &Arc<Connection>,
tx_id: u64,
) -> Result<()> {
let mut sm = mv_store
.commit_tx(tx_id, conn.pager.borrow().clone(), conn)
.unwrap();
let result = sm.step(&mv_store)?;
assert!(sm.is_finalized());
match result {
TransitionResult::Done(()) => Ok(()),
_ => unreachable!(),
}
}
pub(crate) fn commit_tx_no_conn(
db: &MvccTestDbNoConn,
tx_id: u64,
conn: &Arc<Connection>,
) -> Result<(), LimboError> {
let mut sm = db
.db
.get_mv_store()
.unwrap()
.commit_tx(tx_id, conn.pager.borrow().clone(), conn)
.unwrap();
let result = sm.step(db.db.mv_store.as_ref().unwrap())?;
assert!(sm.is_finalized());
match result {
TransitionResult::Done(()) => Ok(()),
_ => unreachable!(),
}
}
#[test]
fn test_lazy_scan_cursor_basic() {
let (db, tx_id) = setup_lazy_db(&[1, 2, 3, 4, 5]);
@@ -801,7 +809,7 @@ fn test_cursor_with_empty_table() {
// FIXME: force page 1 initialization
let pager = db.conn.pager.borrow().clone();
let tx_id = db.mvcc_store.begin_tx(pager.clone());
db.mvcc_store.commit_tx(tx_id, pager, &db.conn).unwrap();
commit_tx(db.mvcc_store.clone(), &db.conn, tx_id).unwrap();
}
let tx_id = db.mvcc_store.begin_tx(db.conn.pager.borrow().clone());
let table_id = 1; // Empty table

View File

@@ -1,13 +0,0 @@
use thiserror::Error;
#[derive(Error, Debug, PartialEq)]
pub enum DatabaseError {
#[error("no such transaction ID: `{0}`")]
NoSuchTransactionID(u64),
#[error("transaction aborted because of a write-write conflict")]
WriteWriteConflict,
#[error("transaction is terminated")]
TxTerminated,
#[error("I/O error: {0}")]
Io(String),
}

View File

@@ -34,7 +34,6 @@
pub mod clock;
pub mod cursor;
pub mod database;
pub mod errors;
pub mod persistent_storage;
pub use clock::LocalClock;
@@ -42,7 +41,9 @@ pub use database::MvStore;
#[cfg(test)]
mod tests {
use crate::mvcc::database::tests::{generate_simple_string_row, MvccTestDbNoConn};
use crate::mvcc::database::tests::{
commit_tx_no_conn, generate_simple_string_row, MvccTestDbNoConn,
};
use crate::mvcc::database::RowID;
use std::sync::atomic::AtomicI64;
use std::sync::atomic::Ordering;
@@ -72,14 +73,10 @@ mod tests {
};
let row = generate_simple_string_row(1, id.row_id, "Hello");
mvcc_store.insert(tx, row.clone()).unwrap();
mvcc_store
.commit_tx(tx, conn.pager.borrow().clone(), &conn)
.unwrap();
commit_tx_no_conn(&db, tx, &conn).unwrap();
let tx = mvcc_store.begin_tx(conn.pager.borrow().clone());
let committed_row = mvcc_store.read(tx, id).unwrap();
mvcc_store
.commit_tx(tx, conn.pager.borrow().clone(), &conn)
.unwrap();
commit_tx_no_conn(&db, tx, &conn).unwrap();
assert_eq!(committed_row, Some(row));
}
})
@@ -97,14 +94,10 @@ mod tests {
};
let row = generate_simple_string_row(1, id.row_id, "World");
mvcc_store.insert(tx, row.clone()).unwrap();
mvcc_store
.commit_tx(tx, conn.pager.borrow().clone(), &conn)
.unwrap();
commit_tx_no_conn(&db, tx, &conn).unwrap();
let tx = mvcc_store.begin_tx(conn.pager.borrow().clone());
let committed_row = mvcc_store.read(tx, id).unwrap();
mvcc_store
.commit_tx(tx, conn.pager.borrow().clone(), &conn)
.unwrap();
commit_tx_no_conn(&db, tx, &conn).unwrap();
assert_eq!(committed_row, Some(row));
}
})
@@ -148,9 +141,7 @@ mod tests {
continue;
}
let committed_row = mvcc_store.read(tx, id).unwrap();
mvcc_store
.commit_tx(tx, conn.pager.borrow().clone(), &conn)
.unwrap();
commit_tx_no_conn(&db, tx, &conn).unwrap();
assert_eq!(committed_row, Some(row));
}
tracing::info!(

View File

@@ -1,7 +1,7 @@
use std::fmt::Debug;
use crate::mvcc::database::{LogRecord, Result};
use crate::mvcc::errors::DatabaseError;
use crate::mvcc::database::LogRecord;
use crate::{LimboError, Result};
#[derive(Debug)]
pub enum Storage {
@@ -24,7 +24,7 @@ impl Storage {
pub fn read_tx_log(&self) -> Result<Vec<LogRecord>> {
match self {
Self::Noop => Err(DatabaseError::Io(
Self::Noop => Err(LimboError::InternalError(
"cannot read from Noop storage".to_string(),
)),
}

81
core/state_machine.rs Normal file
View File

@@ -0,0 +1,81 @@
use crate::Result;
pub enum TransitionResult<Result> {
Io,
Continue,
Done(Result),
}
/// A generic trait for state machines.
pub trait StateTransition {
type State;
type Context;
type SMResult;
/// Transition the state machine to the next state.
///
/// Returns `TransitionResult::Io` if the state machine needs to perform an IO operation.
/// Returns `TransitionResult::Continue` if the state machine needs to continue.
/// Returns `TransitionResult::Done` if the state machine is done.
fn step(&mut self, context: &Self::Context) -> Result<TransitionResult<Self::SMResult>>;
/// Finalize the state machine.
///
/// This is called when the state machine is done.
fn finalize(&mut self, context: &Self::Context) -> Result<()>;
/// Check if the state machine is finalized.
fn is_finalized(&self) -> bool;
}
pub struct StateMachine<State: StateTransition> {
state: State,
is_finalized: bool,
}
/// A generic state machine that loops calling `transition` until it returns `TransitionResult::Done` or `TransitionResult::Io`.
impl<State: StateTransition> StateMachine<State> {
pub fn new(state: State) -> Self {
Self {
state,
is_finalized: false,
}
}
}
impl<State: StateTransition> StateTransition for StateMachine<State> {
type State = State;
type Context = State::Context;
type SMResult = State::SMResult;
fn step(&mut self, context: &Self::Context) -> Result<TransitionResult<Self::SMResult>> {
loop {
if self.is_finalized {
unreachable!("StateMachine::transition: state machine is finalized");
}
match self.state.step(context)? {
TransitionResult::Io => {
return Ok(TransitionResult::Io);
}
TransitionResult::Continue => {
continue;
}
TransitionResult::Done(result) => {
assert!(self.state.is_finalized());
self.is_finalized = true;
return Ok(TransitionResult::Done(result));
}
}
}
}
fn finalize(&mut self, context: &Self::Context) -> Result<()> {
self.state.finalize(context)?;
self.is_finalized = true;
Ok(())
}
fn is_finalized(&self) -> bool {
self.is_finalized
}
}

View File

@@ -27,6 +27,7 @@ pub mod sorter;
use crate::{
error::LimboError,
function::{AggFunc, FuncCtx},
state_machine::StateTransition,
storage::sqlite3_ondisk::SmallVec,
translate::plan::TableReferences,
types::{IOResult, RawSlice, TextRef},
@@ -442,7 +443,12 @@ impl Program {
// FIXME: we don't want to commit stuff from other programs.
let mut mv_transactions = conn.mv_transactions.borrow_mut();
for tx_id in mv_transactions.iter() {
mv_store.commit_tx(*tx_id, pager.clone(), &conn).unwrap();
let mut state_machine =
mv_store.commit_tx(*tx_id, pager.clone(), &conn).unwrap();
state_machine
.step(mv_store)
.map_err(|e| LimboError::InternalError(e.to_string()))?;
assert!(state_machine.is_finalized());
}
mv_transactions.clear();
}