mirror of
https://github.com/aljazceru/turso.git
synced 2026-01-20 08:34:19 +01:00
opfs for sync in one commit!
This commit is contained in:
@@ -10,9 +10,9 @@ use crate::{
|
||||
Result,
|
||||
};
|
||||
|
||||
pub struct DatabaseReplayGenerator<Ctx = ()> {
|
||||
pub struct DatabaseReplayGenerator {
|
||||
pub conn: Arc<turso_core::Connection>,
|
||||
pub opts: DatabaseReplaySessionOpts<Ctx>,
|
||||
pub opts: DatabaseReplaySessionOpts,
|
||||
}
|
||||
|
||||
pub struct ReplayInfo {
|
||||
@@ -24,8 +24,8 @@ pub struct ReplayInfo {
|
||||
}
|
||||
|
||||
const SQLITE_SCHEMA_TABLE: &str = "sqlite_schema";
|
||||
impl<Ctx> DatabaseReplayGenerator<Ctx> {
|
||||
pub fn new(conn: Arc<turso_core::Connection>, opts: DatabaseReplaySessionOpts<Ctx>) -> Self {
|
||||
impl DatabaseReplayGenerator {
|
||||
pub fn new(conn: Arc<turso_core::Connection>, opts: DatabaseReplaySessionOpts) -> Self {
|
||||
Self { conn, opts }
|
||||
}
|
||||
pub fn create_mutation(
|
||||
@@ -118,7 +118,7 @@ impl<Ctx> DatabaseReplayGenerator<Ctx> {
|
||||
}
|
||||
match change {
|
||||
DatabaseChangeType::Delete => {
|
||||
if self.opts.use_implicit_rowid {
|
||||
if self.opts.use_implicit_rowid || info.pk_column_indices.is_none() {
|
||||
vec![turso_core::Value::Integer(id)]
|
||||
} else {
|
||||
let mut values = Vec::new();
|
||||
@@ -168,7 +168,7 @@ impl<Ctx> DatabaseReplayGenerator<Ctx> {
|
||||
}
|
||||
}
|
||||
}
|
||||
pub async fn replay_info(
|
||||
pub async fn replay_info<Ctx>(
|
||||
&self,
|
||||
coro: &Coro<Ctx>,
|
||||
change: &DatabaseTapeRowChange,
|
||||
@@ -276,7 +276,7 @@ impl<Ctx> DatabaseReplayGenerator<Ctx> {
|
||||
}
|
||||
}
|
||||
}
|
||||
pub(crate) async fn update_query(
|
||||
pub(crate) async fn update_query<Ctx>(
|
||||
&self,
|
||||
coro: &Coro<Ctx>,
|
||||
table_name: &str,
|
||||
@@ -320,7 +320,7 @@ impl<Ctx> DatabaseReplayGenerator<Ctx> {
|
||||
is_ddl_replay: false,
|
||||
})
|
||||
}
|
||||
pub(crate) async fn insert_query(
|
||||
pub(crate) async fn insert_query<Ctx>(
|
||||
&self,
|
||||
coro: &Coro<Ctx>,
|
||||
table_name: &str,
|
||||
@@ -371,7 +371,7 @@ impl<Ctx> DatabaseReplayGenerator<Ctx> {
|
||||
is_ddl_replay: false,
|
||||
})
|
||||
}
|
||||
pub(crate) async fn delete_query(
|
||||
pub(crate) async fn delete_query<Ctx>(
|
||||
&self,
|
||||
coro: &Coro<Ctx>,
|
||||
table_name: &str,
|
||||
@@ -406,7 +406,7 @@ impl<Ctx> DatabaseReplayGenerator<Ctx> {
|
||||
})
|
||||
}
|
||||
|
||||
async fn table_columns_info(
|
||||
async fn table_columns_info<Ctx>(
|
||||
&self,
|
||||
coro: &Coro<Ctx>,
|
||||
table_name: &str,
|
||||
|
||||
@@ -1,18 +1,18 @@
|
||||
use std::{
|
||||
cell::RefCell,
|
||||
collections::{HashMap, HashSet},
|
||||
sync::Arc,
|
||||
sync::{Arc, Mutex},
|
||||
};
|
||||
|
||||
use turso_core::OpenFlags;
|
||||
use uuid::Uuid;
|
||||
|
||||
use crate::{
|
||||
database_replay_generator::DatabaseReplayGenerator,
|
||||
database_sync_operations::{
|
||||
bootstrap_db_file, connect_untracked, count_local_changes, fetch_last_change_id, has_table,
|
||||
push_logical_changes, read_wal_salt, reset_wal_file, update_last_change_id, wait_full_body,
|
||||
wal_apply_from_file, wal_pull_to_file, PAGE_SIZE, WAL_FRAME_HEADER, WAL_FRAME_SIZE,
|
||||
acquire_slot, apply_transformation, bootstrap_db_file, connect_untracked,
|
||||
count_local_changes, fetch_last_change_id, has_table, push_logical_changes, read_wal_salt,
|
||||
reset_wal_file, update_last_change_id, wait_all_results, wal_apply_from_file,
|
||||
wal_pull_to_file, PAGE_SIZE, WAL_FRAME_HEADER, WAL_FRAME_SIZE,
|
||||
},
|
||||
database_tape::{
|
||||
DatabaseChangesIteratorMode, DatabaseChangesIteratorOpts, DatabaseReplaySession,
|
||||
@@ -23,34 +23,24 @@ use crate::{
|
||||
io_operations::IoOperations,
|
||||
protocol_io::ProtocolIO,
|
||||
types::{
|
||||
Coro, DatabaseMetadata, DatabasePullRevision, DatabaseSyncEngineProtocolVersion,
|
||||
DatabaseTapeOperation, DbChangesStatus, SyncEngineStats, Transform,
|
||||
Coro, DatabaseMetadata, DatabasePullRevision, DatabaseRowTransformResult,
|
||||
DatabaseSyncEngineProtocolVersion, DatabaseTapeOperation, DbChangesStatus, SyncEngineStats,
|
||||
DATABASE_METADATA_VERSION,
|
||||
},
|
||||
wal_session::WalSession,
|
||||
Result,
|
||||
};
|
||||
|
||||
#[derive(Clone)]
|
||||
pub struct DatabaseSyncEngineOpts<Ctx> {
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct DatabaseSyncEngineOpts {
|
||||
pub client_name: String,
|
||||
pub tables_ignore: Vec<String>,
|
||||
pub transform: Option<Transform<Ctx>>,
|
||||
pub use_transform: bool,
|
||||
pub wal_pull_batch_size: u64,
|
||||
pub protocol_version_hint: DatabaseSyncEngineProtocolVersion,
|
||||
}
|
||||
|
||||
impl<Ctx> std::fmt::Debug for DatabaseSyncEngineOpts<Ctx> {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("DatabaseSyncEngineOpts")
|
||||
.field("client_name", &self.client_name)
|
||||
.field("tables_ignore", &self.tables_ignore)
|
||||
.field("transform.is_some()", &self.transform.is_some())
|
||||
.field("wal_pull_batch_size", &self.wal_pull_batch_size)
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DatabaseSyncEngine<P: ProtocolIO, Ctx> {
|
||||
pub struct DatabaseSyncEngine<P: ProtocolIO> {
|
||||
io: Arc<dyn turso_core::IO>,
|
||||
protocol: Arc<P>,
|
||||
db_file: Arc<dyn turso_core::DatabaseStorage>,
|
||||
@@ -59,7 +49,8 @@ pub struct DatabaseSyncEngine<P: ProtocolIO, Ctx> {
|
||||
revert_db_wal_path: String,
|
||||
main_db_path: String,
|
||||
meta_path: String,
|
||||
opts: DatabaseSyncEngineOpts<Ctx>,
|
||||
changes_file: Arc<Mutex<Option<Arc<dyn turso_core::File>>>>,
|
||||
opts: DatabaseSyncEngineOpts,
|
||||
meta: RefCell<DatabaseMetadata>,
|
||||
client_unique_id: String,
|
||||
}
|
||||
@@ -68,18 +59,19 @@ fn db_size_from_page(page: &[u8]) -> u32 {
|
||||
u32::from_be_bytes(page[28..28 + 4].try_into().unwrap())
|
||||
}
|
||||
|
||||
impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
impl<P: ProtocolIO> DatabaseSyncEngine<P> {
|
||||
/// Creates new instance of SyncEngine and initialize it immediately if no consistent local data exists
|
||||
pub async fn new(
|
||||
pub async fn new<Ctx>(
|
||||
coro: &Coro<Ctx>,
|
||||
io: Arc<dyn turso_core::IO>,
|
||||
protocol: Arc<P>,
|
||||
main_db_path: &str,
|
||||
opts: DatabaseSyncEngineOpts<Ctx>,
|
||||
opts: DatabaseSyncEngineOpts,
|
||||
) -> Result<Self> {
|
||||
let main_db_wal_path = format!("{main_db_path}-wal");
|
||||
let revert_db_wal_path = format!("{main_db_path}-wal-revert");
|
||||
let meta_path = format!("{main_db_path}-info");
|
||||
let changes_path = format!("{main_db_path}-changes");
|
||||
|
||||
let db_file = io.open_file(main_db_path, turso_core::OpenFlags::Create, false)?;
|
||||
let db_file = Arc::new(turso_core::storage::database::DatabaseFile::new(db_file));
|
||||
@@ -87,7 +79,7 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
tracing::info!("init(path={}): opts={:?}", main_db_path, opts);
|
||||
|
||||
let completion = protocol.full_read(&meta_path)?;
|
||||
let data = wait_full_body(coro, &completion).await?;
|
||||
let data = wait_all_results(coro, &completion).await?;
|
||||
let meta = if data.is_empty() {
|
||||
None
|
||||
} else {
|
||||
@@ -107,21 +99,33 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
)
|
||||
.await?;
|
||||
let meta = DatabaseMetadata {
|
||||
version: DATABASE_METADATA_VERSION.to_string(),
|
||||
client_unique_id,
|
||||
synced_revision: Some(revision),
|
||||
revert_since_wal_salt: None,
|
||||
revert_since_wal_watermark: 0,
|
||||
last_pushed_change_id_hint: 0,
|
||||
last_pushed_pull_gen_hint: 0,
|
||||
last_pull_unix_time: io.now().secs,
|
||||
last_push_unix_time: None,
|
||||
};
|
||||
tracing::info!("write meta after successful bootstrap: meta={meta:?}");
|
||||
let completion = protocol.full_write(&meta_path, meta.dump()?)?;
|
||||
// todo: what happen if we will actually update the metadata on disk but fail and so in memory state will not be updated
|
||||
wait_full_body(coro, &completion).await?;
|
||||
wait_all_results(coro, &completion).await?;
|
||||
meta
|
||||
}
|
||||
};
|
||||
|
||||
if meta.version != DATABASE_METADATA_VERSION {
|
||||
return Err(Error::DatabaseSyncEngineError(format!(
|
||||
"unsupported metadata version: {}",
|
||||
meta.version
|
||||
)));
|
||||
}
|
||||
|
||||
tracing::info!("check if main db file exists");
|
||||
|
||||
let main_exists = io.try_open(main_db_path)?.is_some();
|
||||
if !main_exists {
|
||||
let error = "main DB file doesn't exists, but metadata is".to_string();
|
||||
@@ -140,8 +144,9 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
cdc_table: None,
|
||||
cdc_mode: Some("full".to_string()),
|
||||
};
|
||||
let main_tape = DatabaseTape::new_with_opts(main_db, tape_opts);
|
||||
tracing::info!("initialize database tape connection: path={}", main_db_path);
|
||||
let main_tape = DatabaseTape::new_with_opts(main_db, tape_opts);
|
||||
let changes_file = io.open_file(&changes_path, OpenFlags::Create, false)?;
|
||||
let mut db = Self {
|
||||
io,
|
||||
protocol,
|
||||
@@ -151,6 +156,7 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
revert_db_wal_path,
|
||||
main_db_path: main_db_path.to_string(),
|
||||
meta_path: format!("{main_db_path}-info"),
|
||||
changes_file: Arc::new(Mutex::new(Some(changes_file))),
|
||||
opts,
|
||||
meta: RefCell::new(meta.clone()),
|
||||
client_unique_id: meta.client_unique_id.clone(),
|
||||
@@ -163,10 +169,10 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
} = synced_revision
|
||||
{
|
||||
// sync WAL from the remote in case of bootstrap - all subsequent initializations will be fast
|
||||
if let Some(changes) = db.wait_changes_from_remote(coro).await? {
|
||||
db.apply_changes_from_remote(coro, changes).await?;
|
||||
}
|
||||
db.pull_changes_from_remote(coro).await?;
|
||||
}
|
||||
|
||||
tracing::info!("sync engine was initialized");
|
||||
Ok(db)
|
||||
}
|
||||
|
||||
@@ -184,7 +190,10 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
Ok(conn)
|
||||
}
|
||||
|
||||
async fn checkpoint_passive(&mut self, coro: &Coro<Ctx>) -> Result<(Option<Vec<u32>>, u64)> {
|
||||
async fn checkpoint_passive<Ctx>(
|
||||
&mut self,
|
||||
coro: &Coro<Ctx>,
|
||||
) -> Result<(Option<Vec<u32>>, u64)> {
|
||||
let watermark = self.meta().revert_since_wal_watermark;
|
||||
tracing::info!(
|
||||
"checkpoint(path={:?}): revert_since_wal_watermark={}",
|
||||
@@ -232,16 +241,32 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
Ok((main_wal_salt, watermark))
|
||||
}
|
||||
|
||||
pub async fn stats(&self, coro: &Coro<Ctx>) -> Result<SyncEngineStats> {
|
||||
pub async fn stats<Ctx>(&self, coro: &Coro<Ctx>) -> Result<SyncEngineStats> {
|
||||
let main_conn = connect_untracked(&self.main_tape)?;
|
||||
let change_id = self.meta().last_pushed_change_id_hint;
|
||||
let last_pull_unix_time = self.meta().last_pull_unix_time;
|
||||
let last_push_unix_time = self.meta().last_push_unix_time;
|
||||
let revert_wal_path = &self.revert_db_wal_path;
|
||||
let revert_wal_file = self
|
||||
.io
|
||||
.open_file(revert_wal_path, OpenFlags::all(), false)?;
|
||||
let revert_wal_size = revert_wal_file.size()?;
|
||||
let main_wal_frames = main_conn.wal_state()?.max_frame;
|
||||
let main_wal_size = if main_wal_frames == 0 {
|
||||
0
|
||||
} else {
|
||||
WAL_FRAME_HEADER as u64 + WAL_FRAME_SIZE as u64 * main_wal_frames
|
||||
};
|
||||
Ok(SyncEngineStats {
|
||||
cdc_operations: count_local_changes(coro, &main_conn, change_id).await?,
|
||||
wal_size: main_conn.wal_state()?.max_frame as i64,
|
||||
main_wal_size,
|
||||
revert_wal_size,
|
||||
last_pull_unix_time,
|
||||
last_push_unix_time,
|
||||
})
|
||||
}
|
||||
|
||||
pub async fn checkpoint(&mut self, coro: &Coro<Ctx>) -> Result<()> {
|
||||
pub async fn checkpoint<Ctx>(&mut self, coro: &Coro<Ctx>) -> Result<()> {
|
||||
let (main_wal_salt, watermark) = self.checkpoint_passive(coro).await?;
|
||||
|
||||
let main_conn = connect_untracked(&self.main_tape)?;
|
||||
@@ -341,36 +366,32 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn wait_changes_from_remote(
|
||||
&self,
|
||||
coro: &Coro<Ctx>,
|
||||
) -> Result<Option<DbChangesStatus>> {
|
||||
let file_path = format!("{}-frames-{}", self.main_db_path, Uuid::new_v4());
|
||||
tracing::info!(
|
||||
"wait_changes(path={}): file_path={}",
|
||||
self.main_db_path,
|
||||
file_path
|
||||
);
|
||||
let file = self.io.create(&file_path)?;
|
||||
pub async fn wait_changes_from_remote<Ctx>(&self, coro: &Coro<Ctx>) -> Result<DbChangesStatus> {
|
||||
tracing::info!("wait_changes(path={})", self.main_db_path);
|
||||
|
||||
let file = acquire_slot(&self.changes_file)?;
|
||||
|
||||
let now = self.io.now();
|
||||
let revision = self.meta().synced_revision.clone().unwrap();
|
||||
let next_revision = wal_pull_to_file(
|
||||
coro,
|
||||
self.protocol.as_ref(),
|
||||
file.clone(),
|
||||
&file.value,
|
||||
&revision,
|
||||
self.opts.wal_pull_batch_size,
|
||||
)
|
||||
.await?;
|
||||
|
||||
if file.size()? == 0 {
|
||||
if file.value.size()? == 0 {
|
||||
tracing::info!(
|
||||
"wait_changes(path={}): no changes detected, removing changes file {}",
|
||||
self.main_db_path,
|
||||
file_path
|
||||
"wait_changes(path={}): no changes detected",
|
||||
self.main_db_path
|
||||
);
|
||||
self.io.remove_file(&file_path)?;
|
||||
return Ok(None);
|
||||
return Ok(DbChangesStatus {
|
||||
time: now,
|
||||
revision: next_revision,
|
||||
file_slot: None,
|
||||
});
|
||||
}
|
||||
|
||||
tracing::info!(
|
||||
@@ -380,26 +401,25 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
next_revision
|
||||
);
|
||||
|
||||
Ok(Some(DbChangesStatus {
|
||||
Ok(DbChangesStatus {
|
||||
time: now,
|
||||
revision: next_revision,
|
||||
file_path,
|
||||
}))
|
||||
file_slot: Some(file),
|
||||
})
|
||||
}
|
||||
|
||||
/// Sync all new changes from remote DB and apply them locally
|
||||
/// This method will **not** send local changed to the remote
|
||||
/// This method will block writes for the period of pull
|
||||
pub async fn apply_changes_from_remote(
|
||||
pub async fn apply_changes_from_remote<Ctx>(
|
||||
&mut self,
|
||||
coro: &Coro<Ctx>,
|
||||
remote_changes: DbChangesStatus,
|
||||
now: turso_core::Instant,
|
||||
) -> Result<()> {
|
||||
let pull_result = self.apply_changes_internal(coro, &remote_changes).await;
|
||||
let cleanup_result: Result<()> = self
|
||||
.io
|
||||
.remove_file(&remote_changes.file_path)
|
||||
.inspect_err(|e| tracing::error!("failed to cleanup changes file: {e}"))
|
||||
.map_err(|e| e.into());
|
||||
assert!(remote_changes.file_slot.is_some(), "file_slot must be set");
|
||||
let changes_file = remote_changes.file_slot.as_ref().unwrap().value.clone();
|
||||
let pull_result = self.apply_changes_internal(coro, &changes_file).await;
|
||||
let Ok(revert_since_wal_watermark) = pull_result else {
|
||||
return Err(pull_result.err().unwrap());
|
||||
};
|
||||
@@ -411,34 +431,24 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
)?;
|
||||
reset_wal_file(coro, revert_wal_file, 0).await?;
|
||||
|
||||
self.update_meta(coro, |meta| {
|
||||
meta.revert_since_wal_watermark = revert_since_wal_watermark;
|
||||
meta.synced_revision = Some(remote_changes.revision);
|
||||
meta.last_pushed_change_id_hint = 0;
|
||||
self.update_meta(coro, |m| {
|
||||
m.revert_since_wal_watermark = revert_since_wal_watermark;
|
||||
m.synced_revision = Some(remote_changes.revision);
|
||||
m.last_pushed_change_id_hint = 0;
|
||||
m.last_pull_unix_time = now.secs;
|
||||
})
|
||||
.await?;
|
||||
|
||||
cleanup_result
|
||||
Ok(())
|
||||
}
|
||||
async fn apply_changes_internal(
|
||||
async fn apply_changes_internal<Ctx>(
|
||||
&mut self,
|
||||
coro: &Coro<Ctx>,
|
||||
remote_changes: &DbChangesStatus,
|
||||
changes_file: &Arc<dyn turso_core::File>,
|
||||
) -> Result<u64> {
|
||||
tracing::info!(
|
||||
"apply_changes(path={}, changes={:?})",
|
||||
self.main_db_path,
|
||||
remote_changes
|
||||
);
|
||||
tracing::info!("apply_changes(path={})", self.main_db_path);
|
||||
|
||||
let (_, watermark) = self.checkpoint_passive(coro).await?;
|
||||
|
||||
let changes_file = self.io.open_file(
|
||||
&remote_changes.file_path,
|
||||
turso_core::OpenFlags::empty(),
|
||||
false,
|
||||
)?;
|
||||
|
||||
let revert_conn = self.open_revert_db_conn()?;
|
||||
let main_conn = connect_untracked(&self.main_tape)?;
|
||||
|
||||
@@ -476,6 +486,9 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
let mut iterator = self.main_tape.iterate_changes(iterate_opts)?;
|
||||
while let Some(operation) = iterator.next(coro).await? {
|
||||
match operation {
|
||||
DatabaseTapeOperation::StmtReplay(_) => {
|
||||
panic!("changes iterator must not use StmtReplay option")
|
||||
}
|
||||
DatabaseTapeOperation::RowChange(change) => local_changes.push(change),
|
||||
DatabaseTapeOperation::Commit => continue,
|
||||
}
|
||||
@@ -553,18 +566,46 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
cached_insert_stmt: HashMap::new(),
|
||||
cached_update_stmt: HashMap::new(),
|
||||
in_txn: true,
|
||||
generator: DatabaseReplayGenerator::<Ctx> {
|
||||
generator: DatabaseReplayGenerator {
|
||||
conn: main_conn.clone(),
|
||||
opts: DatabaseReplaySessionOpts::<Ctx> {
|
||||
opts: DatabaseReplaySessionOpts {
|
||||
use_implicit_rowid: false,
|
||||
transform: self.opts.transform.clone(),
|
||||
},
|
||||
},
|
||||
};
|
||||
for change in local_changes {
|
||||
let operation = DatabaseTapeOperation::RowChange(change);
|
||||
|
||||
let mut transformed = if self.opts.use_transform {
|
||||
Some(
|
||||
apply_transformation(
|
||||
&coro,
|
||||
self.protocol.as_ref(),
|
||||
&local_changes,
|
||||
&replay.generator,
|
||||
)
|
||||
.await?,
|
||||
)
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
assert!(!replay.conn().get_auto_commit());
|
||||
for (i, change) in local_changes.into_iter().enumerate() {
|
||||
let operation = if let Some(transformed) = &mut transformed {
|
||||
match std::mem::replace(&mut transformed[i], DatabaseRowTransformResult::Skip) {
|
||||
DatabaseRowTransformResult::Keep => {
|
||||
DatabaseTapeOperation::RowChange(change)
|
||||
}
|
||||
DatabaseRowTransformResult::Skip => continue,
|
||||
DatabaseRowTransformResult::Rewrite(replay) => {
|
||||
DatabaseTapeOperation::StmtReplay(replay)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
DatabaseTapeOperation::RowChange(change)
|
||||
};
|
||||
replay.replay(coro, operation).await?;
|
||||
}
|
||||
assert!(!replay.conn().get_auto_commit());
|
||||
|
||||
main_session.wal_session.end(true)?;
|
||||
}
|
||||
@@ -575,7 +616,7 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
/// Sync local changes to remote DB
|
||||
/// This method will **not** pull remote changes to the local DB
|
||||
/// This method will **not** block writes for the period of sync
|
||||
pub async fn push_changes_to_remote(&self, coro: &Coro<Ctx>) -> Result<()> {
|
||||
pub async fn push_changes_to_remote<Ctx>(&self, coro: &Coro<Ctx>) -> Result<()> {
|
||||
tracing::info!("push_changes(path={})", self.main_db_path);
|
||||
|
||||
let (_, change_id) = push_logical_changes(
|
||||
@@ -589,6 +630,7 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
|
||||
self.update_meta(coro, |m| {
|
||||
m.last_pushed_change_id_hint = change_id;
|
||||
m.last_push_unix_time = Some(self.io.now().secs);
|
||||
})
|
||||
.await?;
|
||||
|
||||
@@ -596,7 +638,7 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
}
|
||||
|
||||
/// Create read/write database connection and appropriately configure it before use
|
||||
pub async fn connect_rw(&self, coro: &Coro<Ctx>) -> Result<Arc<turso_core::Connection>> {
|
||||
pub async fn connect_rw<Ctx>(&self, coro: &Coro<Ctx>) -> Result<Arc<turso_core::Connection>> {
|
||||
let conn = self.main_tape.connect(coro).await?;
|
||||
conn.wal_auto_checkpoint_disable();
|
||||
Ok(conn)
|
||||
@@ -604,12 +646,24 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
|
||||
/// Sync local changes to remote DB and bring new changes from remote to local
|
||||
/// This method will block writes for the period of sync
|
||||
pub async fn sync(&mut self, coro: &Coro<Ctx>) -> Result<()> {
|
||||
pub async fn sync<Ctx>(&mut self, coro: &Coro<Ctx>) -> Result<()> {
|
||||
// todo(sivukhin): this is bit suboptimal as both 'push' and 'pull' will call pull_synced_from_remote
|
||||
// but for now - keep it simple
|
||||
self.push_changes_to_remote(coro).await?;
|
||||
if let Some(changes) = self.wait_changes_from_remote(coro).await? {
|
||||
self.apply_changes_from_remote(coro, changes).await?;
|
||||
self.pull_changes_from_remote(coro).await?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn pull_changes_from_remote<Ctx>(&mut self, coro: &Coro<Ctx>) -> Result<()> {
|
||||
let now = self.io.now();
|
||||
let changes = self.wait_changes_from_remote(coro).await?;
|
||||
if changes.file_slot.is_some() {
|
||||
self.apply_changes_from_remote(coro, changes, now).await?;
|
||||
} else {
|
||||
self.update_meta(coro, |m| {
|
||||
m.last_pull_unix_time = now.secs;
|
||||
})
|
||||
.await?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
@@ -618,7 +672,7 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
self.meta.borrow()
|
||||
}
|
||||
|
||||
async fn update_meta(
|
||||
async fn update_meta<Ctx>(
|
||||
&self,
|
||||
coro: &Coro<Ctx>,
|
||||
update: impl FnOnce(&mut DatabaseMetadata),
|
||||
@@ -628,7 +682,7 @@ impl<P: ProtocolIO, Ctx> DatabaseSyncEngine<P, Ctx> {
|
||||
tracing::info!("update_meta: {meta:?}");
|
||||
let completion = self.protocol.full_write(&self.meta_path, meta.dump()?)?;
|
||||
// todo: what happen if we will actually update the metadata on disk but fail and so in memory state will not be updated
|
||||
wait_full_body(coro, &completion).await?;
|
||||
wait_all_results(coro, &completion).await?;
|
||||
self.meta.replace(meta);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
use std::sync::Arc;
|
||||
use std::sync::{Arc, Mutex};
|
||||
|
||||
use bytes::BytesMut;
|
||||
use prost::Message;
|
||||
@@ -22,8 +22,9 @@ use crate::{
|
||||
PullUpdatesRespProtoBody, Stmt, StmtResult, StreamRequest,
|
||||
},
|
||||
types::{
|
||||
Coro, DatabasePullRevision, DatabaseSyncEngineProtocolVersion, DatabaseTapeOperation,
|
||||
DatabaseTapeRowChangeType, DbSyncInfo, DbSyncStatus, ProtocolCommand,
|
||||
Coro, DatabasePullRevision, DatabaseRowTransformResult, DatabaseSyncEngineProtocolVersion,
|
||||
DatabaseTapeOperation, DatabaseTapeRowChange, DatabaseTapeRowChangeType, DbSyncInfo,
|
||||
DbSyncStatus, ProtocolCommand,
|
||||
},
|
||||
wal_session::WalSession,
|
||||
Result,
|
||||
@@ -34,7 +35,30 @@ pub const WAL_FRAME_HEADER: usize = 24;
|
||||
pub const PAGE_SIZE: usize = 4096;
|
||||
pub const WAL_FRAME_SIZE: usize = WAL_FRAME_HEADER + PAGE_SIZE;
|
||||
|
||||
enum WalHttpPullResult<C: DataCompletion> {
|
||||
pub struct MutexSlot<T: Clone> {
|
||||
pub value: T,
|
||||
pub slot: Arc<Mutex<Option<T>>>,
|
||||
}
|
||||
|
||||
impl<T: Clone> Drop for MutexSlot<T> {
|
||||
fn drop(&mut self) {
|
||||
self.slot.lock().unwrap().replace(self.value.clone());
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn acquire_slot<T: Clone>(slot: &Arc<Mutex<Option<T>>>) -> Result<MutexSlot<T>> {
|
||||
let Some(value) = slot.lock().unwrap().take() else {
|
||||
return Err(Error::DatabaseSyncEngineError(
|
||||
"changes file already acquired by another operation".to_string(),
|
||||
));
|
||||
};
|
||||
Ok(MutexSlot {
|
||||
value,
|
||||
slot: slot.clone(),
|
||||
})
|
||||
}
|
||||
|
||||
enum WalHttpPullResult<C: DataCompletion<u8>> {
|
||||
Frames(C),
|
||||
NeedCheckpoint(DbSyncStatus),
|
||||
}
|
||||
@@ -56,10 +80,10 @@ pub async fn db_bootstrap<C: ProtocolIO, Ctx>(
|
||||
client: &C,
|
||||
db: Arc<dyn turso_core::File>,
|
||||
) -> Result<DbSyncInfo> {
|
||||
tracing::debug!("db_bootstrap");
|
||||
tracing::info!("db_bootstrap");
|
||||
let start_time = std::time::Instant::now();
|
||||
let db_info = db_info_http(coro, client).await?;
|
||||
tracing::debug!("db_bootstrap: fetched db_info={db_info:?}");
|
||||
tracing::info!("db_bootstrap: fetched db_info={db_info:?}");
|
||||
let content = db_bootstrap_http(coro, client, db_info.current_generation).await?;
|
||||
let mut pos = 0;
|
||||
loop {
|
||||
@@ -99,21 +123,21 @@ pub async fn db_bootstrap<C: ProtocolIO, Ctx>(
|
||||
}
|
||||
|
||||
let elapsed = std::time::Instant::now().duration_since(start_time);
|
||||
tracing::debug!("db_bootstrap: finished: bytes={pos}, elapsed={:?}", elapsed);
|
||||
tracing::info!("db_bootstrap: finished: bytes={pos}, elapsed={:?}", elapsed);
|
||||
|
||||
Ok(db_info)
|
||||
}
|
||||
|
||||
pub async fn wal_apply_from_file<Ctx>(
|
||||
coro: &Coro<Ctx>,
|
||||
frames_file: Arc<dyn turso_core::File>,
|
||||
frames_file: &Arc<dyn turso_core::File>,
|
||||
session: &mut DatabaseWalSession,
|
||||
) -> Result<u32> {
|
||||
let size = frames_file.size()?;
|
||||
assert!(size % WAL_FRAME_SIZE as u64 == 0);
|
||||
#[allow(clippy::arc_with_non_send_sync)]
|
||||
let buffer = Arc::new(Buffer::new_temporary(WAL_FRAME_SIZE));
|
||||
tracing::debug!("wal_apply_from_file: size={}", size);
|
||||
tracing::info!("wal_apply_from_file: size={}", size);
|
||||
let mut db_size = 0;
|
||||
for offset in (0..size).step_by(WAL_FRAME_SIZE) {
|
||||
let c = Completion::new_read(buffer.clone(), move |result| {
|
||||
@@ -139,10 +163,21 @@ pub async fn wal_apply_from_file<Ctx>(
|
||||
pub async fn wal_pull_to_file<C: ProtocolIO, Ctx>(
|
||||
coro: &Coro<Ctx>,
|
||||
client: &C,
|
||||
frames_file: Arc<dyn turso_core::File>,
|
||||
frames_file: &Arc<dyn turso_core::File>,
|
||||
revision: &DatabasePullRevision,
|
||||
wal_pull_batch_size: u64,
|
||||
) -> Result<DatabasePullRevision> {
|
||||
// truncate file before pulling new data
|
||||
let c = Completion::new_trunc(move |result| {
|
||||
let Ok(rc) = result else {
|
||||
return;
|
||||
};
|
||||
assert!(rc as usize == 0);
|
||||
});
|
||||
let c = frames_file.truncate(0, c)?;
|
||||
while !c.is_completed() {
|
||||
coro.yield_(ProtocolCommand::IO).await?;
|
||||
}
|
||||
match revision {
|
||||
DatabasePullRevision::Legacy {
|
||||
generation,
|
||||
@@ -169,7 +204,7 @@ pub async fn wal_pull_to_file<C: ProtocolIO, Ctx>(
|
||||
pub async fn wal_pull_to_file_v1<C: ProtocolIO, Ctx>(
|
||||
coro: &Coro<Ctx>,
|
||||
client: &C,
|
||||
frames_file: Arc<dyn turso_core::File>,
|
||||
frames_file: &Arc<dyn turso_core::File>,
|
||||
revision: &str,
|
||||
) -> Result<DatabasePullRevision> {
|
||||
tracing::info!("wal_pull: revision={revision}");
|
||||
@@ -263,7 +298,7 @@ pub async fn wal_pull_to_file_v1<C: ProtocolIO, Ctx>(
|
||||
pub async fn wal_pull_to_file_legacy<C: ProtocolIO, Ctx>(
|
||||
coro: &Coro<Ctx>,
|
||||
client: &C,
|
||||
frames_file: Arc<dyn turso_core::File>,
|
||||
frames_file: &Arc<dyn turso_core::File>,
|
||||
mut generation: u64,
|
||||
mut start_frame: u64,
|
||||
wal_pull_batch_size: u64,
|
||||
@@ -284,9 +319,9 @@ pub async fn wal_pull_to_file_legacy<C: ProtocolIO, Ctx>(
|
||||
let data = match result {
|
||||
WalHttpPullResult::NeedCheckpoint(status) => {
|
||||
assert!(status.status == "checkpoint_needed");
|
||||
tracing::debug!("wal_pull: need checkpoint: status={status:?}");
|
||||
tracing::info!("wal_pull: need checkpoint: status={status:?}");
|
||||
if status.generation == generation && status.max_frame_no < start_frame {
|
||||
tracing::debug!("wal_pull: end of history: status={:?}", status);
|
||||
tracing::info!("wal_pull: end of history: status={:?}", status);
|
||||
break DatabasePullRevision::Legacy {
|
||||
generation: status.generation,
|
||||
synced_frame_no: Some(status.max_frame_no),
|
||||
@@ -655,7 +690,7 @@ pub async fn push_logical_changes<C: ProtocolIO, Ctx>(
|
||||
client: &C,
|
||||
source: &DatabaseTape,
|
||||
client_id: &str,
|
||||
opts: &DatabaseSyncEngineOpts<Ctx>,
|
||||
opts: &DatabaseSyncEngineOpts,
|
||||
) -> Result<(i64, i64)> {
|
||||
tracing::info!("push_logical_changes: client_id={client_id}");
|
||||
let source_conn = connect_untracked(source)?;
|
||||
@@ -666,7 +701,6 @@ pub async fn push_logical_changes<C: ProtocolIO, Ctx>(
|
||||
tracing::debug!("push_logical_changes: last_change_id={:?}", last_change_id);
|
||||
let replay_opts = DatabaseReplaySessionOpts {
|
||||
use_implicit_rowid: false,
|
||||
transform: None,
|
||||
};
|
||||
|
||||
let generator = DatabaseReplayGenerator::new(source_conn, replay_opts);
|
||||
@@ -697,15 +731,13 @@ pub async fn push_logical_changes<C: ProtocolIO, Ctx>(
|
||||
];
|
||||
let mut rows_changed = 0;
|
||||
let mut changes = source.iterate_changes(iterate_opts)?;
|
||||
let mut local_changes = Vec::new();
|
||||
while let Some(operation) = changes.next(coro).await? {
|
||||
match operation {
|
||||
DatabaseTapeOperation::StmtReplay(_) => {
|
||||
panic!("changes iterator must not use StmtReplay option")
|
||||
}
|
||||
DatabaseTapeOperation::RowChange(change) => {
|
||||
assert!(
|
||||
last_change_id.is_none() || last_change_id.unwrap() < change.change_id,
|
||||
"change id must be strictly increasing: last_change_id={:?}, change.change_id={}",
|
||||
last_change_id,
|
||||
change.change_id
|
||||
);
|
||||
if change.table_name == TURSO_SYNC_TABLE_NAME {
|
||||
continue;
|
||||
}
|
||||
@@ -713,38 +745,68 @@ pub async fn push_logical_changes<C: ProtocolIO, Ctx>(
|
||||
if ignore.iter().any(|x| &change.table_name == x) {
|
||||
continue;
|
||||
}
|
||||
rows_changed += 1;
|
||||
// we give user full control over CDC table - so let's not emit assert here for now
|
||||
if last_change_id.is_some() && last_change_id.unwrap() + 1 != change.change_id {
|
||||
tracing::warn!(
|
||||
"out of order change sequence: {} -> {}",
|
||||
last_change_id.unwrap(),
|
||||
change.change_id
|
||||
);
|
||||
local_changes.push(change);
|
||||
}
|
||||
DatabaseTapeOperation::Commit => continue,
|
||||
}
|
||||
}
|
||||
|
||||
let mut transformed = if opts.use_transform {
|
||||
Some(apply_transformation(&coro, client, &local_changes, &generator).await?)
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
tracing::info!("local_changes: {:?}", local_changes);
|
||||
|
||||
for (i, change) in local_changes.into_iter().enumerate() {
|
||||
let change_id = change.change_id;
|
||||
let operation = if let Some(transformed) = &mut transformed {
|
||||
match std::mem::replace(&mut transformed[i], DatabaseRowTransformResult::Skip) {
|
||||
DatabaseRowTransformResult::Keep => DatabaseTapeOperation::RowChange(change),
|
||||
DatabaseRowTransformResult::Skip => continue,
|
||||
DatabaseRowTransformResult::Rewrite(replay) => {
|
||||
DatabaseTapeOperation::StmtReplay(replay)
|
||||
}
|
||||
last_change_id = Some(change.change_id);
|
||||
}
|
||||
} else {
|
||||
DatabaseTapeOperation::RowChange(change)
|
||||
};
|
||||
tracing::info!(
|
||||
"change_id: {}, last_change_id: {:?}",
|
||||
change_id,
|
||||
last_change_id
|
||||
);
|
||||
assert!(
|
||||
last_change_id.is_none() || last_change_id.unwrap() < change_id,
|
||||
"change id must be strictly increasing: last_change_id={:?}, change.change_id={}",
|
||||
last_change_id,
|
||||
change_id
|
||||
);
|
||||
rows_changed += 1;
|
||||
// we give user full control over CDC table - so let's not emit assert here for now
|
||||
if last_change_id.is_some() && last_change_id.unwrap() + 1 != change_id {
|
||||
tracing::warn!(
|
||||
"out of order change sequence: {} -> {}",
|
||||
last_change_id.unwrap(),
|
||||
change_id
|
||||
);
|
||||
}
|
||||
last_change_id = Some(change_id);
|
||||
match operation {
|
||||
DatabaseTapeOperation::Commit => {
|
||||
panic!("Commit operation must not be emited at this stage")
|
||||
}
|
||||
DatabaseTapeOperation::StmtReplay(replay) => sql_over_http_requests.push(Stmt {
|
||||
sql: Some(replay.sql),
|
||||
sql_id: None,
|
||||
args: convert_to_args(replay.values),
|
||||
named_args: Vec::new(),
|
||||
want_rows: Some(false),
|
||||
replication_index: None,
|
||||
}),
|
||||
DatabaseTapeOperation::RowChange(change) => {
|
||||
let replay_info = generator.replay_info(coro, &change).await?;
|
||||
if !replay_info.is_ddl_replay {
|
||||
if let Some(transform) = &opts.transform {
|
||||
let mutation = generator.create_mutation(&replay_info, &change)?;
|
||||
if let Some(statement) = transform(&coro.ctx.borrow(), mutation)? {
|
||||
tracing::info!(
|
||||
"push_logical_changes: use mutation from custom transformer: sql={}, values={:?}",
|
||||
statement.sql,
|
||||
statement.values
|
||||
);
|
||||
sql_over_http_requests.push(Stmt {
|
||||
sql: Some(statement.sql),
|
||||
sql_id: None,
|
||||
args: convert_to_args(statement.values),
|
||||
named_args: Vec::new(),
|
||||
want_rows: Some(false),
|
||||
replication_index: None,
|
||||
});
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
let change_type = (&change.change).into();
|
||||
match change.change {
|
||||
DatabaseTapeRowChangeType::Delete { before } => {
|
||||
@@ -825,43 +887,42 @@ pub async fn push_logical_changes<C: ProtocolIO, Ctx>(
|
||||
}
|
||||
}
|
||||
}
|
||||
DatabaseTapeOperation::Commit => {
|
||||
if rows_changed > 0 {
|
||||
tracing::info!("prepare update stmt for turso_sync_last_change_id table with client_id={} and last_change_id={:?}", client_id, last_change_id);
|
||||
// update turso_sync_last_change_id table with new value before commit
|
||||
let next_change_id = last_change_id.unwrap_or(0);
|
||||
tracing::info!("push_logical_changes: client_id={client_id}, set pull_gen={source_pull_gen}, change_id={next_change_id}, rows_changed={rows_changed}");
|
||||
sql_over_http_requests.push(Stmt {
|
||||
sql: Some(TURSO_SYNC_UPSERT_LAST_CHANGE_ID.to_string()),
|
||||
sql_id: None,
|
||||
args: vec![
|
||||
server_proto::Value::Text {
|
||||
value: client_id.to_string(),
|
||||
},
|
||||
server_proto::Value::Integer {
|
||||
value: source_pull_gen,
|
||||
},
|
||||
server_proto::Value::Integer {
|
||||
value: next_change_id,
|
||||
},
|
||||
],
|
||||
named_args: Vec::new(),
|
||||
want_rows: Some(false),
|
||||
replication_index: None,
|
||||
});
|
||||
}
|
||||
sql_over_http_requests.push(Stmt {
|
||||
sql: Some("COMMIT".to_string()),
|
||||
sql_id: None,
|
||||
args: Vec::new(),
|
||||
named_args: Vec::new(),
|
||||
want_rows: Some(false),
|
||||
replication_index: None,
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if rows_changed > 0 {
|
||||
tracing::info!("prepare update stmt for turso_sync_last_change_id table with client_id={} and last_change_id={:?}", client_id, last_change_id);
|
||||
// update turso_sync_last_change_id table with new value before commit
|
||||
let next_change_id = last_change_id.unwrap_or(0);
|
||||
tracing::info!("push_logical_changes: client_id={client_id}, set pull_gen={source_pull_gen}, change_id={next_change_id}, rows_changed={rows_changed}");
|
||||
sql_over_http_requests.push(Stmt {
|
||||
sql: Some(TURSO_SYNC_UPSERT_LAST_CHANGE_ID.to_string()),
|
||||
sql_id: None,
|
||||
args: vec![
|
||||
server_proto::Value::Text {
|
||||
value: client_id.to_string(),
|
||||
},
|
||||
server_proto::Value::Integer {
|
||||
value: source_pull_gen,
|
||||
},
|
||||
server_proto::Value::Integer {
|
||||
value: next_change_id,
|
||||
},
|
||||
],
|
||||
named_args: Vec::new(),
|
||||
want_rows: Some(false),
|
||||
replication_index: None,
|
||||
});
|
||||
}
|
||||
sql_over_http_requests.push(Stmt {
|
||||
sql: Some("COMMIT".to_string()),
|
||||
sql_id: None,
|
||||
args: Vec::new(),
|
||||
named_args: Vec::new(),
|
||||
want_rows: Some(false),
|
||||
replication_index: None,
|
||||
});
|
||||
|
||||
tracing::trace!("hrana request: {:?}", sql_over_http_requests);
|
||||
let replay_hrana_request = server_proto::PipelineReqBody {
|
||||
baton: None,
|
||||
@@ -876,6 +937,30 @@ pub async fn push_logical_changes<C: ProtocolIO, Ctx>(
|
||||
Ok((source_pull_gen, last_change_id.unwrap_or(0)))
|
||||
}
|
||||
|
||||
pub async fn apply_transformation<Ctx, P: ProtocolIO>(
|
||||
coro: &Coro<Ctx>,
|
||||
client: &P,
|
||||
changes: &Vec<DatabaseTapeRowChange>,
|
||||
generator: &DatabaseReplayGenerator,
|
||||
) -> Result<Vec<DatabaseRowTransformResult>> {
|
||||
let mut mutations = Vec::new();
|
||||
for change in changes {
|
||||
let replay_info = generator.replay_info(&coro, &change).await?;
|
||||
mutations.push(generator.create_mutation(&replay_info, &change)?);
|
||||
}
|
||||
let completion = client.transform(mutations)?;
|
||||
let transformed = wait_all_results(&coro, &completion).await?;
|
||||
if transformed.len() != changes.len() {
|
||||
return Err(Error::DatabaseSyncEngineError(format!(
|
||||
"unexpected result from custom transformation: mismatch in shapes: {} != {}",
|
||||
transformed.len(),
|
||||
changes.len()
|
||||
)));
|
||||
}
|
||||
tracing::info!("apply_transformation: got {:?}", transformed);
|
||||
Ok(transformed)
|
||||
}
|
||||
|
||||
pub async fn read_wal_salt<Ctx>(
|
||||
coro: &Coro<Ctx>,
|
||||
wal: &Arc<dyn turso_core::File>,
|
||||
@@ -1102,7 +1187,7 @@ async fn sql_execute_http<C: ProtocolIO, Ctx>(
|
||||
let error = format!("sql_execute_http: unexpected status code: {status}");
|
||||
return Err(Error::DatabaseSyncEngineError(error));
|
||||
}
|
||||
let response = wait_full_body(coro, &completion).await?;
|
||||
let response = wait_all_results(coro, &completion).await?;
|
||||
let response: server_proto::PipelineRespBody = serde_json::from_slice(&response)?;
|
||||
tracing::debug!("hrana response: {:?}", response);
|
||||
let mut results = Vec::new();
|
||||
@@ -1134,7 +1219,7 @@ async fn wal_pull_http<C: ProtocolIO, Ctx>(
|
||||
generation: u64,
|
||||
start_frame: u64,
|
||||
end_frame: u64,
|
||||
) -> Result<WalHttpPullResult<C::DataCompletion>> {
|
||||
) -> Result<WalHttpPullResult<C::DataCompletionBytes>> {
|
||||
let completion = client.http(
|
||||
"GET",
|
||||
&format!("/sync/{generation}/{start_frame}/{end_frame}"),
|
||||
@@ -1143,7 +1228,7 @@ async fn wal_pull_http<C: ProtocolIO, Ctx>(
|
||||
)?;
|
||||
let status = wait_status(coro, &completion).await?;
|
||||
if status == http::StatusCode::BAD_REQUEST {
|
||||
let status_body = wait_full_body(coro, &completion).await?;
|
||||
let status_body = wait_all_results(coro, &completion).await?;
|
||||
let status: DbSyncStatus = serde_json::from_slice(&status_body)?;
|
||||
if status.status == "checkpoint_needed" {
|
||||
return Ok(WalHttpPullResult::NeedCheckpoint(status));
|
||||
@@ -1178,7 +1263,7 @@ async fn wal_push_http<C: ProtocolIO, Ctx>(
|
||||
&[],
|
||||
)?;
|
||||
let status = wait_status(coro, &completion).await?;
|
||||
let status_body = wait_full_body(coro, &completion).await?;
|
||||
let status_body = wait_all_results(coro, &completion).await?;
|
||||
if status != http::StatusCode::OK {
|
||||
let error = std::str::from_utf8(&status_body).ok().unwrap_or("");
|
||||
return Err(Error::DatabaseSyncEngineError(format!(
|
||||
@@ -1191,7 +1276,7 @@ async fn wal_push_http<C: ProtocolIO, Ctx>(
|
||||
async fn db_info_http<C: ProtocolIO, Ctx>(coro: &Coro<Ctx>, client: &C) -> Result<DbSyncInfo> {
|
||||
let completion = client.http("GET", "/info", None, &[])?;
|
||||
let status = wait_status(coro, &completion).await?;
|
||||
let status_body = wait_full_body(coro, &completion).await?;
|
||||
let status_body = wait_all_results(coro, &completion).await?;
|
||||
if status != http::StatusCode::OK {
|
||||
return Err(Error::DatabaseSyncEngineError(format!(
|
||||
"db_info go unexpected status: {status}"
|
||||
@@ -1204,7 +1289,7 @@ async fn db_bootstrap_http<C: ProtocolIO, Ctx>(
|
||||
coro: &Coro<Ctx>,
|
||||
client: &C,
|
||||
generation: u64,
|
||||
) -> Result<C::DataCompletion> {
|
||||
) -> Result<C::DataCompletionBytes> {
|
||||
let completion = client.http("GET", &format!("/export/{generation}"), None, &[])?;
|
||||
let status = wait_status(coro, &completion).await?;
|
||||
if status != http::StatusCode::OK.as_u16() {
|
||||
@@ -1215,7 +1300,10 @@ async fn db_bootstrap_http<C: ProtocolIO, Ctx>(
|
||||
Ok(completion)
|
||||
}
|
||||
|
||||
pub async fn wait_status<Ctx>(coro: &Coro<Ctx>, completion: &impl DataCompletion) -> Result<u16> {
|
||||
pub async fn wait_status<Ctx, T>(
|
||||
coro: &Coro<Ctx>,
|
||||
completion: &impl DataCompletion<T>,
|
||||
) -> Result<u16> {
|
||||
while completion.status()?.is_none() {
|
||||
coro.yield_(ProtocolCommand::IO).await?;
|
||||
}
|
||||
@@ -1244,7 +1332,7 @@ pub fn read_varint(buf: &[u8]) -> Result<Option<(usize, usize)>> {
|
||||
|
||||
pub async fn wait_proto_message<Ctx, T: prost::Message + Default>(
|
||||
coro: &Coro<Ctx>,
|
||||
completion: &impl DataCompletion,
|
||||
completion: &impl DataCompletion<u8>,
|
||||
bytes: &mut BytesMut,
|
||||
) -> Result<Option<T>> {
|
||||
let start_time = std::time::Instant::now();
|
||||
@@ -1281,21 +1369,21 @@ pub async fn wait_proto_message<Ctx, T: prost::Message + Default>(
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn wait_full_body<Ctx>(
|
||||
pub async fn wait_all_results<Ctx, T: Clone>(
|
||||
coro: &Coro<Ctx>,
|
||||
completion: &impl DataCompletion,
|
||||
) -> Result<Vec<u8>> {
|
||||
let mut bytes = Vec::new();
|
||||
completion: &impl DataCompletion<T>,
|
||||
) -> Result<Vec<T>> {
|
||||
let mut results = Vec::new();
|
||||
loop {
|
||||
while let Some(poll) = completion.poll_data()? {
|
||||
bytes.extend_from_slice(poll.data());
|
||||
results.extend_from_slice(poll.data());
|
||||
}
|
||||
if completion.is_done()? {
|
||||
break;
|
||||
}
|
||||
coro.yield_(ProtocolCommand::IO).await?;
|
||||
}
|
||||
Ok(bytes)
|
||||
Ok(results)
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
@@ -1315,7 +1403,7 @@ mod tests {
|
||||
|
||||
struct TestPollResult(Vec<u8>);
|
||||
|
||||
impl DataPollResult for TestPollResult {
|
||||
impl DataPollResult<u8> for TestPollResult {
|
||||
fn data(&self) -> &[u8] {
|
||||
&self.0
|
||||
}
|
||||
@@ -1326,9 +1414,8 @@ mod tests {
|
||||
chunk: usize,
|
||||
}
|
||||
|
||||
impl DataCompletion for TestCompletion {
|
||||
impl DataCompletion<u8> for TestCompletion {
|
||||
type DataPollResult = TestPollResult;
|
||||
|
||||
fn status(&self) -> crate::Result<Option<u16>> {
|
||||
Ok(Some(200))
|
||||
}
|
||||
|
||||
@@ -11,7 +11,7 @@ use crate::{
|
||||
errors::Error,
|
||||
types::{
|
||||
Coro, DatabaseChange, DatabaseTapeOperation, DatabaseTapeRowChange,
|
||||
DatabaseTapeRowChangeType, ProtocolCommand, Transform,
|
||||
DatabaseTapeRowChangeType, ProtocolCommand,
|
||||
},
|
||||
wal_session::WalSession,
|
||||
Result,
|
||||
@@ -169,8 +169,8 @@ impl DatabaseTape {
|
||||
pub async fn start_replay_session<Ctx>(
|
||||
&self,
|
||||
coro: &Coro<Ctx>,
|
||||
opts: DatabaseReplaySessionOpts<Ctx>,
|
||||
) -> Result<DatabaseReplaySession<Ctx>> {
|
||||
opts: DatabaseReplaySessionOpts,
|
||||
) -> Result<DatabaseReplaySession> {
|
||||
tracing::debug!("opening replay session");
|
||||
let conn = self.connect(coro).await?;
|
||||
conn.execute("BEGIN IMMEDIATE")?;
|
||||
@@ -431,16 +431,14 @@ impl DatabaseChangesIterator {
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
pub struct DatabaseReplaySessionOpts<Ctx = ()> {
|
||||
pub struct DatabaseReplaySessionOpts {
|
||||
pub use_implicit_rowid: bool,
|
||||
pub transform: Option<Transform<Ctx>>,
|
||||
}
|
||||
|
||||
impl<Ctx> std::fmt::Debug for DatabaseReplaySessionOpts<Ctx> {
|
||||
impl std::fmt::Debug for DatabaseReplaySessionOpts {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("DatabaseReplaySessionOpts")
|
||||
.field("use_implicit_rowid", &self.use_implicit_rowid)
|
||||
.field("transform_mutation.is_some()", &self.transform.is_some())
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
@@ -450,13 +448,13 @@ pub(crate) struct CachedStmt {
|
||||
info: ReplayInfo,
|
||||
}
|
||||
|
||||
pub struct DatabaseReplaySession<Ctx = ()> {
|
||||
pub struct DatabaseReplaySession {
|
||||
pub(crate) conn: Arc<turso_core::Connection>,
|
||||
pub(crate) cached_delete_stmt: HashMap<String, CachedStmt>,
|
||||
pub(crate) cached_insert_stmt: HashMap<(String, usize), CachedStmt>,
|
||||
pub(crate) cached_update_stmt: HashMap<(String, Vec<bool>), CachedStmt>,
|
||||
pub(crate) in_txn: bool,
|
||||
pub(crate) generator: DatabaseReplayGenerator<Ctx>,
|
||||
pub(crate) generator: DatabaseReplayGenerator,
|
||||
}
|
||||
|
||||
async fn replay_stmt<Ctx>(
|
||||
@@ -472,11 +470,11 @@ async fn replay_stmt<Ctx>(
|
||||
Ok(())
|
||||
}
|
||||
|
||||
impl<Ctx> DatabaseReplaySession<Ctx> {
|
||||
impl DatabaseReplaySession {
|
||||
pub fn conn(&self) -> Arc<turso_core::Connection> {
|
||||
self.conn.clone()
|
||||
}
|
||||
pub async fn replay(
|
||||
pub async fn replay<Ctx>(
|
||||
&mut self,
|
||||
coro: &Coro<Ctx>,
|
||||
operation: DatabaseTapeOperation,
|
||||
@@ -489,6 +487,11 @@ impl<Ctx> DatabaseReplaySession<Ctx> {
|
||||
self.in_txn = false;
|
||||
}
|
||||
}
|
||||
DatabaseTapeOperation::StmtReplay(replay) => {
|
||||
let mut stmt = self.conn.prepare(&replay.sql)?;
|
||||
replay_stmt(coro, &mut stmt, replay.values).await?;
|
||||
return Ok(());
|
||||
}
|
||||
DatabaseTapeOperation::RowChange(change) => {
|
||||
if !self.in_txn {
|
||||
tracing::trace!("replay: start txn for replaying changes");
|
||||
@@ -502,21 +505,6 @@ impl<Ctx> DatabaseReplaySession<Ctx> {
|
||||
let replay_info = self.generator.replay_info(coro, &change).await?;
|
||||
self.conn.execute(replay_info.query.as_str())?;
|
||||
} else {
|
||||
if let Some(transform) = &self.generator.opts.transform {
|
||||
let replay_info = self.generator.replay_info(coro, &change).await?;
|
||||
let mutation = self.generator.create_mutation(&replay_info, &change)?;
|
||||
let statement = transform(&coro.ctx.borrow(), mutation)?;
|
||||
if let Some(statement) = statement {
|
||||
tracing::info!(
|
||||
"replay: use mutation from custom transformer: sql={}, values={:?}",
|
||||
statement.sql,
|
||||
statement.values
|
||||
);
|
||||
let mut stmt = self.conn.prepare(&statement.sql)?;
|
||||
replay_stmt(coro, &mut stmt, statement.values).await?;
|
||||
return Ok(());
|
||||
}
|
||||
}
|
||||
match change.change {
|
||||
DatabaseTapeRowChangeType::Delete { before } => {
|
||||
let key = self.populate_delete_stmt(coro, table).await?;
|
||||
@@ -625,7 +613,7 @@ impl<Ctx> DatabaseReplaySession<Ctx> {
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
async fn populate_delete_stmt<'a>(
|
||||
async fn populate_delete_stmt<'a, Ctx>(
|
||||
&mut self,
|
||||
coro: &Coro<Ctx>,
|
||||
table: &'a str,
|
||||
@@ -640,7 +628,7 @@ impl<Ctx> DatabaseReplaySession<Ctx> {
|
||||
.insert(table.to_string(), CachedStmt { stmt, info });
|
||||
Ok(table)
|
||||
}
|
||||
async fn populate_insert_stmt(
|
||||
async fn populate_insert_stmt<Ctx>(
|
||||
&mut self,
|
||||
coro: &Coro<Ctx>,
|
||||
table: &str,
|
||||
@@ -661,7 +649,7 @@ impl<Ctx> DatabaseReplaySession<Ctx> {
|
||||
.insert(key.clone(), CachedStmt { stmt, info });
|
||||
Ok(key)
|
||||
}
|
||||
async fn populate_update_stmt(
|
||||
async fn populate_update_stmt<Ctx>(
|
||||
&mut self,
|
||||
coro: &Coro<Ctx>,
|
||||
table: &str,
|
||||
@@ -822,7 +810,6 @@ mod tests {
|
||||
{
|
||||
let opts = DatabaseReplaySessionOpts {
|
||||
use_implicit_rowid: true,
|
||||
transform: None,
|
||||
};
|
||||
let mut session = db2.start_replay_session(&coro, opts).await.unwrap();
|
||||
let opts = Default::default();
|
||||
@@ -902,7 +889,6 @@ mod tests {
|
||||
{
|
||||
let opts = DatabaseReplaySessionOpts {
|
||||
use_implicit_rowid: false,
|
||||
transform: None,
|
||||
};
|
||||
let mut session = db2.start_replay_session(&coro, opts).await.unwrap();
|
||||
let opts = Default::default();
|
||||
@@ -973,7 +959,6 @@ mod tests {
|
||||
{
|
||||
let opts = DatabaseReplaySessionOpts {
|
||||
use_implicit_rowid: false,
|
||||
transform: None,
|
||||
};
|
||||
let mut session = db2.start_replay_session(&coro, opts).await.unwrap();
|
||||
let opts = Default::default();
|
||||
@@ -1048,7 +1033,6 @@ mod tests {
|
||||
{
|
||||
let opts = DatabaseReplaySessionOpts {
|
||||
use_implicit_rowid: false,
|
||||
transform: None,
|
||||
};
|
||||
let mut session = db3.start_replay_session(&coro, opts).await.unwrap();
|
||||
|
||||
@@ -1166,7 +1150,6 @@ mod tests {
|
||||
{
|
||||
let opts = DatabaseReplaySessionOpts {
|
||||
use_implicit_rowid: false,
|
||||
transform: None,
|
||||
};
|
||||
let mut session = db2.start_replay_session(&coro, opts).await.unwrap();
|
||||
|
||||
@@ -1252,7 +1235,6 @@ mod tests {
|
||||
{
|
||||
let opts = DatabaseReplaySessionOpts {
|
||||
use_implicit_rowid: false,
|
||||
transform: None,
|
||||
};
|
||||
let mut session = db2.start_replay_session(&coro, opts).await.unwrap();
|
||||
|
||||
@@ -1349,7 +1331,6 @@ mod tests {
|
||||
{
|
||||
let opts = DatabaseReplaySessionOpts {
|
||||
use_implicit_rowid: false,
|
||||
transform: None,
|
||||
};
|
||||
let mut session = db3.start_replay_session(&coro, opts).await.unwrap();
|
||||
|
||||
|
||||
@@ -1,25 +1,33 @@
|
||||
use crate::Result;
|
||||
use crate::{
|
||||
types::{DatabaseRowMutation, DatabaseRowTransformResult},
|
||||
Result,
|
||||
};
|
||||
|
||||
pub trait DataPollResult {
|
||||
fn data(&self) -> &[u8];
|
||||
pub trait DataPollResult<T> {
|
||||
fn data(&self) -> &[T];
|
||||
}
|
||||
|
||||
pub trait DataCompletion {
|
||||
type DataPollResult: DataPollResult;
|
||||
pub trait DataCompletion<T> {
|
||||
type DataPollResult: DataPollResult<T>;
|
||||
fn status(&self) -> Result<Option<u16>>;
|
||||
fn poll_data(&self) -> Result<Option<Self::DataPollResult>>;
|
||||
fn is_done(&self) -> Result<bool>;
|
||||
}
|
||||
|
||||
pub trait ProtocolIO {
|
||||
type DataCompletion: DataCompletion;
|
||||
fn full_read(&self, path: &str) -> Result<Self::DataCompletion>;
|
||||
fn full_write(&self, path: &str, content: Vec<u8>) -> Result<Self::DataCompletion>;
|
||||
type DataCompletionBytes: DataCompletion<u8>;
|
||||
type DataCompletionTransform: DataCompletion<DatabaseRowTransformResult>;
|
||||
fn full_read(&self, path: &str) -> Result<Self::DataCompletionBytes>;
|
||||
fn full_write(&self, path: &str, content: Vec<u8>) -> Result<Self::DataCompletionBytes>;
|
||||
fn transform(
|
||||
&self,
|
||||
mutations: Vec<DatabaseRowMutation>,
|
||||
) -> Result<Self::DataCompletionTransform>;
|
||||
fn http(
|
||||
&self,
|
||||
method: &str,
|
||||
path: &str,
|
||||
body: Option<Vec<u8>>,
|
||||
headers: &[(&str, &str)],
|
||||
) -> Result<Self::DataCompletion>;
|
||||
) -> Result<Self::DataCompletionBytes>;
|
||||
}
|
||||
|
||||
@@ -2,10 +2,7 @@ use std::{cell::RefCell, collections::HashMap, sync::Arc};
|
||||
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use crate::{errors::Error, Result};
|
||||
|
||||
pub type Transform<Ctx> =
|
||||
Arc<dyn Fn(&Ctx, DatabaseRowMutation) -> Result<Option<DatabaseRowStatement>> + 'static>;
|
||||
use crate::{database_sync_operations::MutexSlot, errors::Error, Result};
|
||||
|
||||
pub struct Coro<Ctx> {
|
||||
pub ctx: RefCell<Ctx>,
|
||||
@@ -48,15 +45,28 @@ pub struct DbSyncStatus {
|
||||
pub max_frame_no: u64,
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct DbChangesStatus {
|
||||
pub time: turso_core::Instant,
|
||||
pub revision: DatabasePullRevision,
|
||||
pub file_path: String,
|
||||
pub file_slot: Option<MutexSlot<Arc<dyn turso_core::File>>>,
|
||||
}
|
||||
|
||||
impl std::fmt::Debug for DbChangesStatus {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("DbChangesStatus")
|
||||
.field("time", &self.time)
|
||||
.field("revision", &self.revision)
|
||||
.field("file_slot.is_some()", &self.file_slot.is_some())
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
pub struct SyncEngineStats {
|
||||
pub cdc_operations: i64,
|
||||
pub wal_size: i64,
|
||||
pub main_wal_size: u64,
|
||||
pub revert_wal_size: u64,
|
||||
pub last_pull_unix_time: i64,
|
||||
pub last_push_unix_time: Option<i64>,
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone, Copy, PartialEq)]
|
||||
@@ -66,8 +76,11 @@ pub enum DatabaseChangeType {
|
||||
Insert,
|
||||
}
|
||||
|
||||
pub const DATABASE_METADATA_VERSION: &str = "v1";
|
||||
|
||||
#[derive(Serialize, Deserialize, Debug, Clone, PartialEq)]
|
||||
pub struct DatabaseMetadata {
|
||||
pub version: String,
|
||||
/// Unique identifier of the client - generated on sync startup
|
||||
pub client_unique_id: String,
|
||||
/// Latest generation from remote which was pulled locally to the Synced DB
|
||||
@@ -75,6 +88,10 @@ pub struct DatabaseMetadata {
|
||||
/// pair of frame_no for Draft and Synced DB such that content of the database file up to these frames is identical
|
||||
pub revert_since_wal_salt: Option<Vec<u32>>,
|
||||
pub revert_since_wal_watermark: u64,
|
||||
/// Unix time of last successful pull
|
||||
pub last_pull_unix_time: i64,
|
||||
/// Unix time of last successful push
|
||||
pub last_push_unix_time: Option<i64>,
|
||||
pub last_pushed_pull_gen_hint: i64,
|
||||
pub last_pushed_change_id_hint: i64,
|
||||
}
|
||||
@@ -269,11 +286,20 @@ pub struct DatabaseRowMutation {
|
||||
pub updates: Option<HashMap<String, turso_core::Value>>,
|
||||
}
|
||||
|
||||
pub struct DatabaseRowStatement {
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct DatabaseStatementReplay {
|
||||
pub sql: String,
|
||||
pub values: Vec<turso_core::Value>,
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone)]
|
||||
pub enum DatabaseRowTransformResult {
|
||||
Keep,
|
||||
Skip,
|
||||
Rewrite(DatabaseStatementReplay),
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
pub enum DatabaseTapeRowChangeType {
|
||||
Delete {
|
||||
before: Vec<turso_core::Value>,
|
||||
@@ -304,12 +330,13 @@ impl From<&DatabaseTapeRowChangeType> for DatabaseChangeType {
|
||||
/// by consuming events from [crate::database_tape::DatabaseChangesIterator]
|
||||
#[derive(Debug)]
|
||||
pub enum DatabaseTapeOperation {
|
||||
StmtReplay(DatabaseStatementReplay),
|
||||
RowChange(DatabaseTapeRowChange),
|
||||
Commit,
|
||||
}
|
||||
|
||||
/// [DatabaseTapeRowChange] is the specific operation over single row which can be performed on database
|
||||
#[derive(Debug)]
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct DatabaseTapeRowChange {
|
||||
pub change_id: i64,
|
||||
pub change_time: u64,
|
||||
|
||||
Reference in New Issue
Block a user