From 0917f84cde2d272141eaa79d933ed0736fb668e5 Mon Sep 17 00:00:00 2001 From: ad hoc Date: Fri, 2 Aug 2024 23:54:55 +0200 Subject: [PATCH 01/13] introduce NamespaceConfigurator --- .../src/namespace/configurator/mod.rs | 56 ++++++ .../src/namespace/configurator/primary.rs | 128 ++++++++++++ .../src/namespace/configurator/replica.rs | 190 ++++++++++++++++++ libsql-server/src/namespace/mod.rs | 17 +- 4 files changed, 383 insertions(+), 8 deletions(-) create mode 100644 libsql-server/src/namespace/configurator/mod.rs create mode 100644 libsql-server/src/namespace/configurator/primary.rs create mode 100644 libsql-server/src/namespace/configurator/replica.rs diff --git a/libsql-server/src/namespace/configurator/mod.rs b/libsql-server/src/namespace/configurator/mod.rs new file mode 100644 index 0000000000..0caa1de149 --- /dev/null +++ b/libsql-server/src/namespace/configurator/mod.rs @@ -0,0 +1,56 @@ +use std::pin::Pin; + +use futures::Future; + +use super::broadcasters::BroadcasterHandle; +use super::meta_store::MetaStoreHandle; +use super::{NamespaceConfig, NamespaceName, NamespaceStore, ResetCb, ResolveNamespacePathFn, RestoreOption}; + +mod replica; +mod primary; + +type DynConfigurator = Box; + +#[derive(Default)] +struct NamespaceConfigurators { + replica_configurator: Option, + primary_configurator: Option, + schema_configurator: Option, +} + +impl NamespaceConfigurators { + pub fn with_primary( + &mut self, + c: impl ConfigureNamespace + Send + Sync + 'static, + ) -> &mut Self { + self.primary_configurator = Some(Box::new(c)); + self + } + + pub fn with_replica( + &mut self, + c: impl ConfigureNamespace + Send + Sync + 'static, + ) -> &mut Self { + self.replica_configurator = Some(Box::new(c)); + self + } + + pub fn with_schema(&mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> &mut Self { + self.schema_configurator = Some(Box::new(c)); + self + } +} + +pub trait ConfigureNamespace { + fn setup<'a>( + &'a self, + ns_config: &'a NamespaceConfig, + db_config: MetaStoreHandle, + restore_option: RestoreOption, + name: &'a NamespaceName, + reset: ResetCb, + resolve_attach_path: ResolveNamespacePathFn, + store: NamespaceStore, + broadcaster: BroadcasterHandle, + ) -> Pin> + Send + 'a>>; +} diff --git a/libsql-server/src/namespace/configurator/primary.rs b/libsql-server/src/namespace/configurator/primary.rs new file mode 100644 index 0000000000..f28d288a97 --- /dev/null +++ b/libsql-server/src/namespace/configurator/primary.rs @@ -0,0 +1,128 @@ +use std::sync::atomic::{AtomicBool, Ordering}; +use std::{path::Path, pin::Pin, sync::Arc}; + +use futures::prelude::Future; +use tokio::task::JoinSet; + +use crate::connection::MakeConnection; +use crate::database::{Database, PrimaryDatabase}; +use crate::namespace::{Namespace, NamespaceConfig, NamespaceName, NamespaceStore, ResetCb, ResolveNamespacePathFn, RestoreOption}; +use crate::namespace::meta_store::MetaStoreHandle; +use crate::namespace::broadcasters::BroadcasterHandle; +use crate::run_periodic_checkpoint; +use crate::schema::{has_pending_migration_task, setup_migration_table}; + +use super::ConfigureNamespace; + +pub struct PrimaryConfigurator; + +impl ConfigureNamespace for PrimaryConfigurator { + fn setup<'a>( + &'a self, + config: &'a NamespaceConfig, + meta_store_handle: MetaStoreHandle, + restore_option: RestoreOption, + name: &'a NamespaceName, + _reset: ResetCb, + resolve_attach_path: ResolveNamespacePathFn, + _store: NamespaceStore, + broadcaster: BroadcasterHandle, + ) -> Pin> + Send + 'a>> + { + Box::pin(async move { + let db_path: Arc = config.base_path.join("dbs").join(name.as_str()).into(); + let fresh_namespace = !db_path.try_exists()?; + // FIXME: make that truly atomic. explore the idea of using temp directories, and it's implications + match try_new_primary( + config, + name.clone(), + meta_store_handle, + restore_option, + resolve_attach_path, + db_path.clone(), + broadcaster, + ) + .await + { + Ok(this) => Ok(this), + Err(e) if fresh_namespace => { + tracing::error!("an error occured while deleting creating namespace, cleaning..."); + if let Err(e) = tokio::fs::remove_dir_all(&db_path).await { + tracing::error!("failed to remove dirty namespace directory: {e}") + } + Err(e) + } + Err(e) => Err(e), + } + }) + } +} + +#[tracing::instrument(skip_all, fields(namespace))] +async fn try_new_primary( + ns_config: &NamespaceConfig, + namespace: NamespaceName, + meta_store_handle: MetaStoreHandle, + restore_option: RestoreOption, + resolve_attach_path: ResolveNamespacePathFn, + db_path: Arc, + broadcaster: BroadcasterHandle, +) -> crate::Result { + let mut join_set = JoinSet::new(); + + tokio::fs::create_dir_all(&db_path).await?; + + let block_writes = Arc::new(AtomicBool::new(false)); + let (connection_maker, wal_wrapper, stats) = Namespace::make_primary_connection_maker( + ns_config, + &meta_store_handle, + &db_path, + &namespace, + restore_option, + block_writes.clone(), + &mut join_set, + resolve_attach_path, + broadcaster, + ) + .await?; + let connection_maker = Arc::new(connection_maker); + + if meta_store_handle.get().shared_schema_name.is_some() { + let block_writes = block_writes.clone(); + let conn = connection_maker.create().await?; + tokio::task::spawn_blocking(move || { + conn.with_raw(|conn| -> crate::Result<()> { + setup_migration_table(conn)?; + if has_pending_migration_task(conn)? { + block_writes.store(true, Ordering::SeqCst); + } + Ok(()) + }) + }) + .await + .unwrap()?; + } + + if let Some(checkpoint_interval) = ns_config.checkpoint_interval { + join_set.spawn(run_periodic_checkpoint( + connection_maker.clone(), + checkpoint_interval, + namespace.clone(), + )); + } + + tracing::debug!("Done making new primary"); + + Ok(Namespace { + tasks: join_set, + db: Database::Primary(PrimaryDatabase { + wal_wrapper, + connection_maker, + block_writes, + }), + name: namespace, + stats, + db_config_store: meta_store_handle, + path: db_path.into(), + }) +} diff --git a/libsql-server/src/namespace/configurator/replica.rs b/libsql-server/src/namespace/configurator/replica.rs new file mode 100644 index 0000000000..4d3ca1dadf --- /dev/null +++ b/libsql-server/src/namespace/configurator/replica.rs @@ -0,0 +1,190 @@ +use std::pin::Pin; +use std::sync::Arc; + +use futures::Future; +use libsql_replication::rpc::replication::replication_log_client::ReplicationLogClient; +use tokio::task::JoinSet; + +use crate::connection::write_proxy::MakeWriteProxyConn; +use crate::connection::MakeConnection; +use crate::database::{Database, ReplicaDatabase}; +use crate::namespace::broadcasters::BroadcasterHandle; +use crate::namespace::meta_store::MetaStoreHandle; +use crate::namespace::{Namespace, RestoreOption}; +use crate::namespace::{ + make_stats, NamespaceConfig, NamespaceName, NamespaceStore, ResetCb, ResetOp, + ResolveNamespacePathFn, +}; +use crate::{DB_CREATE_TIMEOUT, DEFAULT_AUTO_CHECKPOINT}; + +use super::ConfigureNamespace; + +pub struct ReplicaConfigurator; + +impl ConfigureNamespace for ReplicaConfigurator { + fn setup<'a>( + &'a self, + config: &'a NamespaceConfig, + meta_store_handle: MetaStoreHandle, + restore_option: RestoreOption, + name: &'a NamespaceName, + reset: ResetCb, + resolve_attach_path: ResolveNamespacePathFn, + store: NamespaceStore, + broadcaster: BroadcasterHandle, + ) -> Pin> + Send + 'a>> + { + Box::pin(async move { + tracing::debug!("creating replica namespace"); + let db_path = config.base_path.join("dbs").join(name.as_str()); + let channel = config.channel.clone().expect("bad replica config"); + let uri = config.uri.clone().expect("bad replica config"); + + let rpc_client = ReplicationLogClient::with_origin(channel.clone(), uri.clone()); + let client = crate::replication::replicator_client::Client::new( + name.clone(), + rpc_client, + &db_path, + meta_store_handle.clone(), + store.clone(), + ) + .await?; + let applied_frame_no_receiver = client.current_frame_no_notifier.subscribe(); + let mut replicator = libsql_replication::replicator::Replicator::new( + client, + db_path.join("data"), + DEFAULT_AUTO_CHECKPOINT, + config.encryption_config.clone(), + ) + .await?; + + tracing::debug!("try perform handshake"); + // force a handshake now, to retrieve the primary's current replication index + match replicator.try_perform_handshake().await { + Err(libsql_replication::replicator::Error::Meta( + libsql_replication::meta::Error::LogIncompatible, + )) => { + tracing::error!( + "trying to replicate incompatible logs, reseting replica and nuking db dir" + ); + std::fs::remove_dir_all(&db_path).unwrap(); + return self.setup( + config, + meta_store_handle, + restore_option, + name, + reset, + resolve_attach_path, + store, + broadcaster, + ) + .await; + } + Err(e) => Err(e)?, + Ok(_) => (), + } + + tracing::debug!("done performing handshake"); + + let primary_current_replicatio_index = replicator.client_mut().primary_replication_index; + + let mut join_set = JoinSet::new(); + let namespace = name.clone(); + join_set.spawn(async move { + use libsql_replication::replicator::Error; + loop { + match replicator.run().await { + err @ Error::Fatal(_) => Err(err)?, + err @ Error::NamespaceDoesntExist => { + tracing::error!("namespace {namespace} doesn't exist, destroying..."); + (reset)(ResetOp::Destroy(namespace.clone())); + Err(err)?; + } + e @ Error::Injector(_) => { + tracing::error!("potential corruption detected while replicating, reseting replica: {e}"); + (reset)(ResetOp::Reset(namespace.clone())); + Err(e)?; + }, + Error::Meta(err) => { + use libsql_replication::meta::Error; + match err { + Error::LogIncompatible => { + tracing::error!("trying to replicate incompatible logs, reseting replica"); + (reset)(ResetOp::Reset(namespace.clone())); + Err(err)?; + } + Error::InvalidMetaFile + | Error::Io(_) + | Error::InvalidLogId + | Error::FailedToCommit(_) + | Error::InvalidReplicationPath + | Error::RequiresCleanDatabase => { + // We retry from last frame index? + tracing::warn!("non-fatal replication error, retrying from last commit index: {err}"); + }, + } + } + e @ (Error::Internal(_) + | Error::Client(_) + | Error::PrimaryHandshakeTimeout + | Error::NeedSnapshot) => { + tracing::warn!("non-fatal replication error, retrying from last commit index: {e}"); + }, + Error::NoHandshake => { + // not strictly necessary, but in case the handshake error goes uncaught, + // we reset the client state. + replicator.client_mut().reset_token(); + } + Error::SnapshotPending => unreachable!(), + } + } + }); + + let stats = make_stats( + &db_path, + &mut join_set, + meta_store_handle.clone(), + config.stats_sender.clone(), + name.clone(), + applied_frame_no_receiver.clone(), + config.encryption_config.clone(), + ) + .await?; + + let connection_maker = MakeWriteProxyConn::new( + db_path.clone(), + config.extensions.clone(), + channel.clone(), + uri.clone(), + stats.clone(), + broadcaster, + meta_store_handle.clone(), + applied_frame_no_receiver, + config.max_response_size, + config.max_total_response_size, + primary_current_replicatio_index, + config.encryption_config.clone(), + resolve_attach_path, + config.make_wal_manager.clone(), + ) + .await? + .throttled( + config.max_concurrent_connections.clone(), + Some(DB_CREATE_TIMEOUT), + config.max_total_response_size, + config.max_concurrent_requests, + ); + + Ok(Namespace { + tasks: join_set, + db: Database::Replica(ReplicaDatabase { + connection_maker: Arc::new(connection_maker), + }), + name: name.clone(), + stats, + db_config_store: meta_store_handle, + path: db_path.into(), + }) + }) + } +} diff --git a/libsql-server/src/namespace/mod.rs b/libsql-server/src/namespace/mod.rs index 6e48e7f1d8..6a04b11fb8 100644 --- a/libsql-server/src/namespace/mod.rs +++ b/libsql-server/src/namespace/mod.rs @@ -1,11 +1,3 @@ -pub mod broadcasters; -mod fork; -pub mod meta_store; -mod name; -pub mod replication_wal; -mod schema_lock; -mod store; - use std::path::{Path, PathBuf}; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::{Arc, Weak}; @@ -57,6 +49,15 @@ pub use self::name::NamespaceName; use self::replication_wal::{make_replication_wal_wrapper, ReplicationWalWrapper}; pub use self::store::NamespaceStore; +pub mod broadcasters; +mod fork; +pub mod meta_store; +mod name; +pub mod replication_wal; +mod schema_lock; +mod store; +mod configurator; + pub type ResetCb = Box; pub type ResolveNamespacePathFn = Arc crate::Result> + Sync + Send + 'static>; From f9daa9e08f58efdebd52acb4d45435266bec34af Mon Sep 17 00:00:00 2001 From: ad hoc Date: Sat, 3 Aug 2024 00:00:45 +0200 Subject: [PATCH 02/13] add configurators to namespace store --- libsql-server/src/namespace/configurator/mod.rs | 2 +- libsql-server/src/namespace/store.rs | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/libsql-server/src/namespace/configurator/mod.rs b/libsql-server/src/namespace/configurator/mod.rs index 0caa1de149..a692f75652 100644 --- a/libsql-server/src/namespace/configurator/mod.rs +++ b/libsql-server/src/namespace/configurator/mod.rs @@ -12,7 +12,7 @@ mod primary; type DynConfigurator = Box; #[derive(Default)] -struct NamespaceConfigurators { +pub(crate) struct NamespaceConfigurators { replica_configurator: Option, primary_configurator: Option, schema_configurator: Option, diff --git a/libsql-server/src/namespace/store.rs b/libsql-server/src/namespace/store.rs index e0147fc2e8..984a520154 100644 --- a/libsql-server/src/namespace/store.rs +++ b/libsql-server/src/namespace/store.rs @@ -19,6 +19,7 @@ use crate::namespace::{NamespaceBottomlessDbId, NamespaceBottomlessDbIdInit, Nam use crate::stats::Stats; use super::broadcasters::{BroadcasterHandle, BroadcasterRegistry}; +use super::configurator::NamespaceConfigurators; use super::meta_store::{MetaStore, MetaStoreHandle}; use super::schema_lock::SchemaLocksRegistry; use super::{Namespace, NamespaceConfig, ResetCb, ResetOp, ResolveNamespacePathFn, RestoreOption}; @@ -47,6 +48,7 @@ pub struct NamespaceStoreInner { pub config: NamespaceConfig, schema_locks: SchemaLocksRegistry, broadcasters: BroadcasterRegistry, + configurators: NamespaceConfigurators, } impl NamespaceStore { @@ -90,6 +92,7 @@ impl NamespaceStore { config, schema_locks: Default::default(), broadcasters: Default::default(), + configurators: NamespaceConfigurators::default(), }), }) } From 8b377a6e06dfc051bedb68976577adb8de339f40 Mon Sep 17 00:00:00 2001 From: ad hoc Date: Sat, 3 Aug 2024 21:18:51 +0200 Subject: [PATCH 03/13] add shcema configurator --- .../src/namespace/configurator/schema.rs | 65 +++++++++++++++++++ 1 file changed, 65 insertions(+) create mode 100644 libsql-server/src/namespace/configurator/schema.rs diff --git a/libsql-server/src/namespace/configurator/schema.rs b/libsql-server/src/namespace/configurator/schema.rs new file mode 100644 index 0000000000..864b75239f --- /dev/null +++ b/libsql-server/src/namespace/configurator/schema.rs @@ -0,0 +1,65 @@ +use std::sync::{atomic::AtomicBool, Arc}; + +use futures::prelude::Future; +use tokio::task::JoinSet; + +use crate::database::{Database, SchemaDatabase}; +use crate::namespace::meta_store::MetaStoreHandle; +use crate::namespace::{ + Namespace, NamespaceConfig, NamespaceName, NamespaceStore, + ResetCb, ResolveNamespacePathFn, RestoreOption, +}; +use crate::namespace::broadcasters::BroadcasterHandle; + +use super::ConfigureNamespace; + +pub struct SchemaConfigurator; + +impl ConfigureNamespace for SchemaConfigurator { + fn setup<'a>( + &'a self, + ns_config: &'a NamespaceConfig, + db_config: MetaStoreHandle, + restore_option: RestoreOption, + name: &'a NamespaceName, + _reset: ResetCb, + resolve_attach_path: ResolveNamespacePathFn, + _store: NamespaceStore, + broadcaster: BroadcasterHandle, + ) -> std::pin::Pin> + Send + 'a>> { + Box::pin(async move { + let mut join_set = JoinSet::new(); + let db_path = ns_config.base_path.join("dbs").join(name.as_str()); + + tokio::fs::create_dir_all(&db_path).await?; + + let (connection_maker, wal_manager, stats) = Namespace::make_primary_connection_maker( + ns_config, + &db_config, + &db_path, + &name, + restore_option, + Arc::new(AtomicBool::new(false)), // this is always false for schema + &mut join_set, + resolve_attach_path, + broadcaster, + ) + .await?; + + Ok(Namespace { + db: Database::Schema(SchemaDatabase::new( + ns_config.migration_scheduler.clone(), + name.clone(), + connection_maker, + wal_manager, + db_config.clone(), + )), + name: name.clone(), + tasks: join_set, + stats, + db_config_store: db_config.clone(), + path: db_path.into(), + }) + }) + } +} From 978dd7147d0304397f261d918c70c2806eff82a5 Mon Sep 17 00:00:00 2001 From: ad hoc Date: Sat, 3 Aug 2024 21:19:00 +0200 Subject: [PATCH 04/13] instanciate namesapces from configurators --- .../src/namespace/configurator/mod.rs | 25 +- libsql-server/src/namespace/fork.rs | 26 +- libsql-server/src/namespace/mod.rs | 374 +----------------- libsql-server/src/namespace/store.rs | 76 ++-- 4 files changed, 74 insertions(+), 427 deletions(-) diff --git a/libsql-server/src/namespace/configurator/mod.rs b/libsql-server/src/namespace/configurator/mod.rs index a692f75652..d3cd390b34 100644 --- a/libsql-server/src/namespace/configurator/mod.rs +++ b/libsql-server/src/namespace/configurator/mod.rs @@ -8,14 +8,19 @@ use super::{NamespaceConfig, NamespaceName, NamespaceStore, ResetCb, ResolveName mod replica; mod primary; +mod schema; -type DynConfigurator = Box; +pub use replica::ReplicaConfigurator; +pub use primary::PrimaryConfigurator; +pub use schema::SchemaConfigurator; + +type DynConfigurator = dyn ConfigureNamespace + Send + Sync + 'static; #[derive(Default)] pub(crate) struct NamespaceConfigurators { - replica_configurator: Option, - primary_configurator: Option, - schema_configurator: Option, + replica_configurator: Option>, + primary_configurator: Option>, + schema_configurator: Option>, } impl NamespaceConfigurators { @@ -39,6 +44,18 @@ impl NamespaceConfigurators { self.schema_configurator = Some(Box::new(c)); self } + + pub fn configure_schema(&self) -> crate::Result<&DynConfigurator> { + self.schema_configurator.as_deref().ok_or_else(|| todo!()) + } + + pub fn configure_primary(&self) -> crate::Result<&DynConfigurator> { + self.primary_configurator.as_deref().ok_or_else(|| todo!()) + } + + pub fn configure_replica(&self) -> crate::Result<&DynConfigurator> { + self.replica_configurator.as_deref().ok_or_else(|| todo!()) + } } pub trait ConfigureNamespace { diff --git a/libsql-server/src/namespace/fork.rs b/libsql-server/src/namespace/fork.rs index dfa053b43d..f25bf7a9a9 100644 --- a/libsql-server/src/namespace/fork.rs +++ b/libsql-server/src/namespace/fork.rs @@ -12,14 +12,12 @@ use tokio::io::{AsyncSeekExt, AsyncWriteExt}; use tokio::time::Duration; use tokio_stream::StreamExt; -use crate::namespace::ResolveNamespacePathFn; use crate::replication::primary::frame_stream::FrameStream; use crate::replication::{LogReadError, ReplicationLogger}; use crate::{BLOCKING_RT, LIBSQL_PAGE_SIZE}; -use super::broadcasters::BroadcasterHandle; use super::meta_store::MetaStoreHandle; -use super::{Namespace, NamespaceConfig, NamespaceName, NamespaceStore, RestoreOption}; +use super::{NamespaceName, NamespaceStore, RestoreOption}; type Result = crate::Result; @@ -54,16 +52,13 @@ async fn write_frame(frame: &FrameBorrowed, temp_file: &mut tokio::fs::File) -> Ok(()) } -pub struct ForkTask<'a> { +pub struct ForkTask { pub base_path: Arc, pub logger: Arc, pub to_namespace: NamespaceName, pub to_config: MetaStoreHandle, pub restore_to: Option, - pub ns_config: &'a NamespaceConfig, - pub resolve_attach: ResolveNamespacePathFn, pub store: NamespaceStore, - pub broadcaster: BroadcasterHandle, } pub struct PointInTimeRestore { @@ -71,7 +66,7 @@ pub struct PointInTimeRestore { pub replicator_options: bottomless::replicator::Options, } -impl<'a> ForkTask<'a> { +impl ForkTask { pub async fn fork(self) -> Result { let base_path = self.base_path.clone(); let dest_namespace = self.to_namespace.clone(); @@ -105,18 +100,9 @@ impl<'a> ForkTask<'a> { let dest_path = self.base_path.join("dbs").join(self.to_namespace.as_str()); tokio::fs::rename(temp_dir.path(), dest_path).await?; - Namespace::from_config( - self.ns_config, - self.to_config.clone(), - RestoreOption::Latest, - &self.to_namespace, - Box::new(|_op| {}), - self.resolve_attach.clone(), - self.store.clone(), - self.broadcaster, - ) - .await - .map_err(|e| ForkError::CreateNamespace(Box::new(e))) + self.store.make_namespace(&self.to_namespace, self.to_config, RestoreOption::Latest) + .await + .map_err(|e| ForkError::CreateNamespace(Box::new(e))) } /// Restores the database state from a local log file. diff --git a/libsql-server/src/namespace/mod.rs b/libsql-server/src/namespace/mod.rs index 6a04b11fb8..41bb3ab9cc 100644 --- a/libsql-server/src/namespace/mod.rs +++ b/libsql-server/src/namespace/mod.rs @@ -1,5 +1,5 @@ use std::path::{Path, PathBuf}; -use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::atomic::AtomicBool; use std::sync::{Arc, Weak}; use anyhow::{Context as _, Error}; @@ -10,7 +10,6 @@ use chrono::NaiveDateTime; use enclose::enclose; use futures_core::{Future, Stream}; use hyper::Uri; -use libsql_replication::rpc::replication::replication_log_client::ReplicationLogClient; use libsql_sys::wal::Sqlite3WalManager; use libsql_sys::EncryptionConfig; use tokio::io::AsyncBufReadExt; @@ -25,20 +24,17 @@ use crate::auth::parse_jwt_keys; use crate::connection::config::DatabaseConfig; use crate::connection::connection_manager::InnerWalManager; use crate::connection::libsql::{open_conn, MakeLibSqlConn}; -use crate::connection::write_proxy::MakeWriteProxyConn; -use crate::connection::Connection; -use crate::connection::MakeConnection; +use crate::connection::{Connection as _, MakeConnection}; use crate::database::{ - Database, DatabaseKind, PrimaryConnection, PrimaryConnectionMaker, PrimaryDatabase, - ReplicaDatabase, SchemaDatabase, + Database, DatabaseKind, PrimaryConnection, PrimaryConnectionMaker, }; use crate::error::LoadDumpError; use crate::replication::script_backup_manager::ScriptBackupManager; use crate::replication::{FrameNo, ReplicationLogger}; -use crate::schema::{has_pending_migration_task, setup_migration_table, SchedulerHandle}; +use crate::schema::SchedulerHandle; use crate::stats::Stats; use crate::{ - run_periodic_checkpoint, StatsSender, BLOCKING_RT, DB_CREATE_TIMEOUT, DEFAULT_AUTO_CHECKPOINT, + StatsSender, BLOCKING_RT, DB_CREATE_TIMEOUT, DEFAULT_AUTO_CHECKPOINT, }; pub use fork::ForkError; @@ -101,54 +97,6 @@ pub struct Namespace { } impl Namespace { - async fn from_config( - ns_config: &NamespaceConfig, - db_config: MetaStoreHandle, - restore_option: RestoreOption, - name: &NamespaceName, - reset: ResetCb, - resolve_attach_path: ResolveNamespacePathFn, - store: NamespaceStore, - broadcaster: BroadcasterHandle, - ) -> crate::Result { - match ns_config.db_kind { - DatabaseKind::Primary if db_config.get().is_shared_schema => { - Self::new_schema( - ns_config, - name.clone(), - db_config, - restore_option, - resolve_attach_path, - broadcaster, - ) - .await - } - DatabaseKind::Primary => { - Self::new_primary( - ns_config, - name.clone(), - db_config, - restore_option, - resolve_attach_path, - broadcaster, - ) - .await - } - DatabaseKind::Replica => { - Self::new_replica( - ns_config, - name.clone(), - db_config, - reset, - resolve_attach_path, - store, - broadcaster, - ) - .await - } - } - } - pub(crate) fn name(&self) -> &NamespaceName { &self.name } @@ -248,40 +196,6 @@ impl Namespace { self.db_config_store.changed() } - async fn new_primary( - config: &NamespaceConfig, - name: NamespaceName, - meta_store_handle: MetaStoreHandle, - restore_option: RestoreOption, - resolve_attach_path: ResolveNamespacePathFn, - broadcaster: BroadcasterHandle, - ) -> crate::Result { - let db_path: Arc = config.base_path.join("dbs").join(name.as_str()).into(); - let fresh_namespace = !db_path.try_exists()?; - // FIXME: make that truly atomic. explore the idea of using temp directories, and it's implications - match Self::try_new_primary( - config, - name.clone(), - meta_store_handle, - restore_option, - resolve_attach_path, - db_path.clone(), - broadcaster, - ) - .await - { - Ok(this) => Ok(this), - Err(e) if fresh_namespace => { - tracing::error!("an error occured while deleting creating namespace, cleaning..."); - if let Err(e) = tokio::fs::remove_dir_all(&db_path).await { - tracing::error!("failed to remove dirty namespace directory: {e}") - } - Err(e) - } - Err(e) => Err(e), - } - } - #[tracing::instrument(skip_all)] async fn make_primary_connection_maker( ns_config: &NamespaceConfig, @@ -417,237 +331,6 @@ impl Namespace { Ok((connection_maker, wal_wrapper, stats)) } - #[tracing::instrument(skip_all, fields(namespace))] - async fn try_new_primary( - ns_config: &NamespaceConfig, - namespace: NamespaceName, - meta_store_handle: MetaStoreHandle, - restore_option: RestoreOption, - resolve_attach_path: ResolveNamespacePathFn, - db_path: Arc, - broadcaster: BroadcasterHandle, - ) -> crate::Result { - let mut join_set = JoinSet::new(); - - tokio::fs::create_dir_all(&db_path).await?; - - let block_writes = Arc::new(AtomicBool::new(false)); - let (connection_maker, wal_wrapper, stats) = Self::make_primary_connection_maker( - ns_config, - &meta_store_handle, - &db_path, - &namespace, - restore_option, - block_writes.clone(), - &mut join_set, - resolve_attach_path, - broadcaster, - ) - .await?; - let connection_maker = Arc::new(connection_maker); - - if meta_store_handle.get().shared_schema_name.is_some() { - let block_writes = block_writes.clone(); - let conn = connection_maker.create().await?; - tokio::task::spawn_blocking(move || { - conn.with_raw(|conn| -> crate::Result<()> { - setup_migration_table(conn)?; - if has_pending_migration_task(conn)? { - block_writes.store(true, Ordering::SeqCst); - } - Ok(()) - }) - }) - .await - .unwrap()?; - } - - if let Some(checkpoint_interval) = ns_config.checkpoint_interval { - join_set.spawn(run_periodic_checkpoint( - connection_maker.clone(), - checkpoint_interval, - namespace.clone(), - )); - } - - tracing::debug!("Done making new primary"); - - Ok(Self { - tasks: join_set, - db: Database::Primary(PrimaryDatabase { - wal_wrapper, - connection_maker, - block_writes, - }), - name: namespace, - stats, - db_config_store: meta_store_handle, - path: db_path.into(), - }) - } - - #[tracing::instrument(skip_all, fields(name))] - #[async_recursion::async_recursion] - async fn new_replica( - config: &NamespaceConfig, - name: NamespaceName, - meta_store_handle: MetaStoreHandle, - reset: ResetCb, - resolve_attach_path: ResolveNamespacePathFn, - store: NamespaceStore, - broadcaster: BroadcasterHandle, - ) -> crate::Result { - tracing::debug!("creating replica namespace"); - let db_path = config.base_path.join("dbs").join(name.as_str()); - let channel = config.channel.clone().expect("bad replica config"); - let uri = config.uri.clone().expect("bad replica config"); - - let rpc_client = ReplicationLogClient::with_origin(channel.clone(), uri.clone()); - let client = crate::replication::replicator_client::Client::new( - name.clone(), - rpc_client, - &db_path, - meta_store_handle.clone(), - store.clone(), - ) - .await?; - let applied_frame_no_receiver = client.current_frame_no_notifier.subscribe(); - let mut replicator = libsql_replication::replicator::Replicator::new( - client, - db_path.join("data"), - DEFAULT_AUTO_CHECKPOINT, - config.encryption_config.clone(), - ) - .await?; - - tracing::debug!("try perform handshake"); - // force a handshake now, to retrieve the primary's current replication index - match replicator.try_perform_handshake().await { - Err(libsql_replication::replicator::Error::Meta( - libsql_replication::meta::Error::LogIncompatible, - )) => { - tracing::error!( - "trying to replicate incompatible logs, reseting replica and nuking db dir" - ); - std::fs::remove_dir_all(&db_path).unwrap(); - return Self::new_replica( - config, - name, - meta_store_handle, - reset, - resolve_attach_path, - store, - broadcaster, - ) - .await; - } - Err(e) => Err(e)?, - Ok(_) => (), - } - - tracing::debug!("done performing handshake"); - - let primary_current_replicatio_index = replicator.client_mut().primary_replication_index; - - let mut join_set = JoinSet::new(); - let namespace = name.clone(); - join_set.spawn(async move { - use libsql_replication::replicator::Error; - loop { - match replicator.run().await { - err @ Error::Fatal(_) => Err(err)?, - err @ Error::NamespaceDoesntExist => { - tracing::error!("namespace {namespace} doesn't exist, destroying..."); - (reset)(ResetOp::Destroy(namespace.clone())); - Err(err)?; - } - e @ Error::Injector(_) => { - tracing::error!("potential corruption detected while replicating, reseting replica: {e}"); - (reset)(ResetOp::Reset(namespace.clone())); - Err(e)?; - }, - Error::Meta(err) => { - use libsql_replication::meta::Error; - match err { - Error::LogIncompatible => { - tracing::error!("trying to replicate incompatible logs, reseting replica"); - (reset)(ResetOp::Reset(namespace.clone())); - Err(err)?; - } - Error::InvalidMetaFile - | Error::Io(_) - | Error::InvalidLogId - | Error::FailedToCommit(_) - | Error::InvalidReplicationPath - | Error::RequiresCleanDatabase => { - // We retry from last frame index? - tracing::warn!("non-fatal replication error, retrying from last commit index: {err}"); - }, - } - } - e @ (Error::Internal(_) - | Error::Client(_) - | Error::PrimaryHandshakeTimeout - | Error::NeedSnapshot) => { - tracing::warn!("non-fatal replication error, retrying from last commit index: {e}"); - }, - Error::NoHandshake => { - // not strictly necessary, but in case the handshake error goes uncaught, - // we reset the client state. - replicator.client_mut().reset_token(); - } - Error::SnapshotPending => unreachable!(), - } - } - }); - - let stats = make_stats( - &db_path, - &mut join_set, - meta_store_handle.clone(), - config.stats_sender.clone(), - name.clone(), - applied_frame_no_receiver.clone(), - config.encryption_config.clone(), - ) - .await?; - - let connection_maker = MakeWriteProxyConn::new( - db_path.clone(), - config.extensions.clone(), - channel.clone(), - uri.clone(), - stats.clone(), - broadcaster, - meta_store_handle.clone(), - applied_frame_no_receiver, - config.max_response_size, - config.max_total_response_size, - primary_current_replicatio_index, - config.encryption_config.clone(), - resolve_attach_path, - config.make_wal_manager.clone(), - ) - .await? - .throttled( - config.max_concurrent_connections.clone(), - Some(DB_CREATE_TIMEOUT), - config.max_total_response_size, - config.max_concurrent_requests, - ); - - Ok(Self { - tasks: join_set, - db: Database::Replica(ReplicaDatabase { - connection_maker: Arc::new(connection_maker), - }), - name, - stats, - db_config_store: meta_store_handle, - path: db_path.into(), - }) - } - async fn fork( ns_config: &NamespaceConfig, from_ns: &Namespace, @@ -655,9 +338,7 @@ impl Namespace { to_ns: NamespaceName, to_config: MetaStoreHandle, timestamp: Option, - resolve_attach: ResolveNamespacePathFn, store: NamespaceStore, - broadcaster: BroadcasterHandle, ) -> crate::Result { let from_config = from_config.get(); match ns_config.db_kind { @@ -696,10 +377,7 @@ impl Namespace { logger, restore_to, to_config, - ns_config, - resolve_attach, store, - broadcaster: broadcaster.handle(to_ns), }; let ns = fork_task.fork().await?; @@ -708,48 +386,6 @@ impl Namespace { DatabaseKind::Replica => Err(ForkError::ForkReplica.into()), } } - - async fn new_schema( - ns_config: &NamespaceConfig, - name: NamespaceName, - meta_store_handle: MetaStoreHandle, - restore_option: RestoreOption, - resolve_attach_path: ResolveNamespacePathFn, - broadcaster: BroadcasterHandle, - ) -> crate::Result { - let mut join_set = JoinSet::new(); - let db_path = ns_config.base_path.join("dbs").join(name.as_str()); - - tokio::fs::create_dir_all(&db_path).await?; - - let (connection_maker, wal_manager, stats) = Self::make_primary_connection_maker( - ns_config, - &meta_store_handle, - &db_path, - &name, - restore_option, - Arc::new(AtomicBool::new(false)), // this is always false for schema - &mut join_set, - resolve_attach_path, - broadcaster, - ) - .await?; - - Ok(Namespace { - db: Database::Schema(SchemaDatabase::new( - ns_config.migration_scheduler.clone(), - name.clone(), - connection_maker, - wal_manager, - meta_store_handle.clone(), - )), - name, - tasks: join_set, - stats, - db_config_store: meta_store_handle, - path: db_path.into(), - }) - } } pub struct NamespaceConfig { diff --git a/libsql-server/src/namespace/store.rs b/libsql-server/src/namespace/store.rs index 984a520154..5a94a7f8eb 100644 --- a/libsql-server/src/namespace/store.rs +++ b/libsql-server/src/namespace/store.rs @@ -13,8 +13,10 @@ use tokio_stream::wrappers::BroadcastStream; use crate::auth::Authenticated; use crate::broadcaster::BroadcastMsg; use crate::connection::config::DatabaseConfig; +use crate::database::DatabaseKind; use crate::error::Error; use crate::metrics::NAMESPACE_LOAD_LATENCY; +use crate::namespace::configurator::{PrimaryConfigurator, ReplicaConfigurator, SchemaConfigurator}; use crate::namespace::{NamespaceBottomlessDbId, NamespaceBottomlessDbIdInit, NamespaceName}; use crate::stats::Stats; @@ -82,6 +84,12 @@ impl NamespaceStore { .time_to_idle(Duration::from_secs(86400)) .build(); + let mut configurators = NamespaceConfigurators::default(); + configurators + .with_primary(PrimaryConfigurator) + .with_replica(ReplicaConfigurator) + .with_schema(SchemaConfigurator); + Ok(Self { inner: Arc::new(NamespaceStoreInner { store, @@ -92,7 +100,7 @@ impl NamespaceStore { config, schema_locks: Default::default(), broadcasters: Default::default(), - configurators: NamespaceConfigurators::default(), + configurators, }), }) } @@ -177,27 +185,17 @@ impl NamespaceStore { ns.destroy().await?; } - let handle = self.inner.metadata.handle(namespace.clone()); + let db_config = self.inner.metadata.handle(namespace.clone()); // destroy on-disk database Namespace::cleanup( &self.inner.config, &namespace, - &handle.get(), + &db_config.get(), false, NamespaceBottomlessDbIdInit::FetchFromConfig, ) .await?; - let ns = Namespace::from_config( - &self.inner.config, - handle, - restore_option, - &namespace, - self.make_reset_cb(), - self.resolve_attach_fn(), - self.clone(), - self.broadcaster(namespace.clone()), - ) - .await?; + let ns = self.make_namespace(&namespace, db_config, restore_option).await?; lock.replace(ns); @@ -304,9 +302,7 @@ impl NamespaceStore { to.clone(), handle.clone(), timestamp, - self.resolve_attach_fn(), self.clone(), - self.broadcaster(to), ) .await?; @@ -381,30 +377,42 @@ impl NamespaceStore { .clone() } + pub(crate) async fn make_namespace( + &self, + namespace: &NamespaceName, + config: MetaStoreHandle, + restore_option: RestoreOption, + ) -> crate::Result { + let configurator = match self.inner.config.db_kind { + DatabaseKind::Primary if config.get().is_shared_schema => { + self.inner.configurators.configure_schema()? + } + DatabaseKind::Primary => self.inner.configurators.configure_primary()?, + DatabaseKind::Replica => self.inner.configurators.configure_replica()?, + }; + let ns = configurator.setup( + &self.inner.config, + config, + restore_option, + namespace, + self.make_reset_cb(), + self.resolve_attach_fn(), + self.clone(), + self.broadcaster(namespace.clone()), + ).await?; + + Ok(ns) + } + async fn load_namespace( &self, namespace: &NamespaceName, db_config: MetaStoreHandle, restore_option: RestoreOption, ) -> crate::Result { - let init = { - let namespace = namespace.clone(); - async move { - let ns = Namespace::from_config( - &self.inner.config, - db_config, - restore_option, - &namespace, - self.make_reset_cb(), - self.resolve_attach_fn(), - self.clone(), - self.broadcaster(namespace.clone()), - ) - .await?; - tracing::info!("loaded namespace: `{namespace}`"); - - Ok(Some(ns)) - } + let init = async { + let ns = self.make_namespace(namespace, db_config, restore_option).await?; + Ok(Some(ns)) }; let before_load = Instant::now(); From 907f2f9381783b09254e605473455c72e97cd40b Mon Sep 17 00:00:00 2001 From: ad hoc Date: Mon, 5 Aug 2024 11:14:40 +0200 Subject: [PATCH 05/13] pass configurators to NamespaceStore::new --- libsql-server/src/lib.rs | 5 +++ .../src/namespace/configurator/mod.rs | 38 ++++++++++++------- libsql-server/src/namespace/mod.rs | 2 +- libsql-server/src/namespace/store.rs | 10 +---- libsql-server/src/schema/scheduler.rs | 38 ++++++++++++++----- 5 files changed, 62 insertions(+), 31 deletions(-) diff --git a/libsql-server/src/lib.rs b/libsql-server/src/lib.rs index 5404a11108..3d816d6bc3 100644 --- a/libsql-server/src/lib.rs +++ b/libsql-server/src/lib.rs @@ -60,6 +60,7 @@ use utils::services::idle_shutdown::IdleShutdownKicker; use self::config::MetaStoreConfig; use self::connection::connection_manager::InnerWalManager; +use self::namespace::configurator::NamespaceConfigurators; use self::namespace::NamespaceStore; use self::net::AddrIncoming; use self::replication::script_backup_manager::{CommandHandler, ScriptBackupManager}; @@ -488,12 +489,16 @@ where meta_store_wal_manager, ) .await?; + + let configurators = NamespaceConfigurators::default(); + let namespace_store: NamespaceStore = NamespaceStore::new( db_kind.is_replica(), self.db_config.snapshot_at_shutdown, self.max_active_namespaces, ns_config, meta_store, + configurators, ) .await?; diff --git a/libsql-server/src/namespace/configurator/mod.rs b/libsql-server/src/namespace/configurator/mod.rs index d3cd390b34..a240c3e410 100644 --- a/libsql-server/src/namespace/configurator/mod.rs +++ b/libsql-server/src/namespace/configurator/mod.rs @@ -4,43 +4,55 @@ use futures::Future; use super::broadcasters::BroadcasterHandle; use super::meta_store::MetaStoreHandle; -use super::{NamespaceConfig, NamespaceName, NamespaceStore, ResetCb, ResolveNamespacePathFn, RestoreOption}; +use super::{ + NamespaceConfig, NamespaceName, NamespaceStore, ResetCb, ResolveNamespacePathFn, RestoreOption, +}; -mod replica; mod primary; +mod replica; mod schema; -pub use replica::ReplicaConfigurator; pub use primary::PrimaryConfigurator; +pub use replica::ReplicaConfigurator; pub use schema::SchemaConfigurator; type DynConfigurator = dyn ConfigureNamespace + Send + Sync + 'static; -#[derive(Default)] pub(crate) struct NamespaceConfigurators { replica_configurator: Option>, primary_configurator: Option>, schema_configurator: Option>, } +impl Default for NamespaceConfigurators { + fn default() -> Self { + Self::empty() + .with_primary(PrimaryConfigurator) + .with_replica(ReplicaConfigurator) + .with_schema(SchemaConfigurator) + } +} + impl NamespaceConfigurators { - pub fn with_primary( - &mut self, - c: impl ConfigureNamespace + Send + Sync + 'static, - ) -> &mut Self { + pub fn empty() -> Self { + Self { + replica_configurator: None, + primary_configurator: None, + schema_configurator: None, + } + } + + pub fn with_primary(mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> Self { self.primary_configurator = Some(Box::new(c)); self } - pub fn with_replica( - &mut self, - c: impl ConfigureNamespace + Send + Sync + 'static, - ) -> &mut Self { + pub fn with_replica(mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> Self { self.replica_configurator = Some(Box::new(c)); self } - pub fn with_schema(&mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> &mut Self { + pub fn with_schema(mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> Self { self.schema_configurator = Some(Box::new(c)); self } diff --git a/libsql-server/src/namespace/mod.rs b/libsql-server/src/namespace/mod.rs index 41bb3ab9cc..5ccda74c54 100644 --- a/libsql-server/src/namespace/mod.rs +++ b/libsql-server/src/namespace/mod.rs @@ -52,7 +52,7 @@ mod name; pub mod replication_wal; mod schema_lock; mod store; -mod configurator; +pub(crate) mod configurator; pub type ResetCb = Box; pub type ResolveNamespacePathFn = diff --git a/libsql-server/src/namespace/store.rs b/libsql-server/src/namespace/store.rs index 5a94a7f8eb..fbce8cd78b 100644 --- a/libsql-server/src/namespace/store.rs +++ b/libsql-server/src/namespace/store.rs @@ -16,7 +16,6 @@ use crate::connection::config::DatabaseConfig; use crate::database::DatabaseKind; use crate::error::Error; use crate::metrics::NAMESPACE_LOAD_LATENCY; -use crate::namespace::configurator::{PrimaryConfigurator, ReplicaConfigurator, SchemaConfigurator}; use crate::namespace::{NamespaceBottomlessDbId, NamespaceBottomlessDbIdInit, NamespaceName}; use crate::stats::Stats; @@ -54,12 +53,13 @@ pub struct NamespaceStoreInner { } impl NamespaceStore { - pub async fn new( + pub(crate) async fn new( allow_lazy_creation: bool, snapshot_at_shutdown: bool, max_active_namespaces: usize, config: NamespaceConfig, metadata: MetaStore, + configurators: NamespaceConfigurators, ) -> crate::Result { tracing::trace!("Max active namespaces: {max_active_namespaces}"); let store = Cache::::builder() @@ -84,12 +84,6 @@ impl NamespaceStore { .time_to_idle(Duration::from_secs(86400)) .build(); - let mut configurators = NamespaceConfigurators::default(); - configurators - .with_primary(PrimaryConfigurator) - .with_replica(ReplicaConfigurator) - .with_schema(SchemaConfigurator); - Ok(Self { inner: Arc::new(NamespaceStoreInner { store, diff --git a/libsql-server/src/schema/scheduler.rs b/libsql-server/src/schema/scheduler.rs index 17fdfb3143..17ce655064 100644 --- a/libsql-server/src/schema/scheduler.rs +++ b/libsql-server/src/schema/scheduler.rs @@ -808,6 +808,9 @@ mod test { use crate::connection::config::DatabaseConfig; use crate::database::DatabaseKind; + use crate::namespace::configurator::{ + NamespaceConfigurators, PrimaryConfigurator, SchemaConfigurator, + }; use crate::namespace::meta_store::{metastore_connection_maker, MetaStore}; use crate::namespace::{NamespaceConfig, RestoreOption}; use crate::schema::SchedulerHandle; @@ -826,9 +829,16 @@ mod test { .unwrap(); let (sender, mut receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new(false, false, 10, config, meta_store) - .await - .unwrap(); + let store = NamespaceStore::new( + false, + false, + 10, + config, + meta_store, + NamespaceConfigurators::default(), + ) + .await + .unwrap(); let mut scheduler = Scheduler::new(store.clone(), maker().unwrap()) .await .unwrap(); @@ -936,9 +946,16 @@ mod test { .unwrap(); let (sender, mut receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new(false, false, 10, config, meta_store) - .await - .unwrap(); + let store = NamespaceStore::new( + false, + false, + 10, + config, + meta_store, + NamespaceConfigurators::default(), + ) + .await + .unwrap(); let mut scheduler = Scheduler::new(store.clone(), maker().unwrap()) .await .unwrap(); @@ -1012,7 +1029,7 @@ mod test { .unwrap(); let (sender, _receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new(false, false, 10, config, meta_store) + let store = NamespaceStore::new(false, false, 10, config, meta_store, NamespaceConfigurators::default()) .await .unwrap(); @@ -1039,7 +1056,10 @@ mod test { .unwrap(); let (sender, mut receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new(false, false, 10, config, meta_store) + let configurators = NamespaceConfigurators::default() + .with_schema(SchemaConfigurator) + .with_primary(PrimaryConfigurator); + let store = NamespaceStore::new(false, false, 10, config, meta_store, configurators) .await .unwrap(); let mut scheduler = Scheduler::new(store.clone(), maker().unwrap()) @@ -1112,7 +1132,7 @@ mod test { .unwrap(); let (sender, _receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new(false, false, 10, config, meta_store) + let store = NamespaceStore::new(false, false, 10, config, meta_store, NamespaceConfigurators::default()) .await .unwrap(); let scheduler = Scheduler::new(store.clone(), maker().unwrap()) From fd03144bcc30e7b85f07abf4630ba9ea58a87d11 Mon Sep 17 00:00:00 2001 From: ad hoc Date: Mon, 5 Aug 2024 15:26:34 +0200 Subject: [PATCH 06/13] decoupled namespace configurators --- libsql-server/src/error.rs | 2 +- libsql-server/src/lib.rs | 86 ++- .../src/namespace/{ => configurator}/fork.rs | 62 +- .../src/namespace/configurator/helpers.rs | 451 ++++++++++++++ .../src/namespace/configurator/mod.rs | 67 ++- .../src/namespace/configurator/primary.rs | 249 +++++--- .../src/namespace/configurator/replica.rs | 140 +++-- .../src/namespace/configurator/schema.rs | 71 ++- libsql-server/src/namespace/mod.rs | 551 +----------------- libsql-server/src/namespace/store.rs | 103 ++-- libsql-server/src/schema/scheduler.rs | 80 ++- 11 files changed, 1056 insertions(+), 806 deletions(-) rename libsql-server/src/namespace/{ => configurator}/fork.rs (77%) create mode 100644 libsql-server/src/namespace/configurator/helpers.rs diff --git a/libsql-server/src/error.rs b/libsql-server/src/error.rs index 371630abdf..9cd0b81485 100644 --- a/libsql-server/src/error.rs +++ b/libsql-server/src/error.rs @@ -4,7 +4,7 @@ use tonic::metadata::errors::InvalidMetadataValueBytes; use crate::{ auth::AuthError, - namespace::{ForkError, NamespaceName}, + namespace::{configurator::fork::ForkError, NamespaceName}, query_result_builder::QueryResultBuilderError, }; diff --git a/libsql-server/src/lib.rs b/libsql-server/src/lib.rs index 3d816d6bc3..8bd3ea4fac 100644 --- a/libsql-server/src/lib.rs +++ b/libsql-server/src/lib.rs @@ -46,7 +46,7 @@ use libsql_wal::registry::WalRegistry; use libsql_wal::storage::NoStorage; use libsql_wal::wal::LibsqlWalManager; use namespace::meta_store::MetaStoreHandle; -use namespace::{NamespaceConfig, NamespaceName}; +use namespace::NamespaceName; use net::Connector; use once_cell::sync::Lazy; use rusqlite::ffi::SQLITE_CONFIG_MALLOC; @@ -60,7 +60,7 @@ use utils::services::idle_shutdown::IdleShutdownKicker; use self::config::MetaStoreConfig; use self::connection::connection_manager::InnerWalManager; -use self::namespace::configurator::NamespaceConfigurators; +use self::namespace::configurator::{BaseNamespaceConfig, NamespaceConfigurators, PrimaryConfigurator, PrimaryExtraConfig, ReplicaConfigurator, SchemaConfigurator}; use self::namespace::NamespaceStore; use self::net::AddrIncoming; use self::replication::script_backup_manager::{CommandHandler, ScriptBackupManager}; @@ -425,11 +425,6 @@ where let user_auth_strategy = self.user_api_config.auth_strategy.clone(); let service_shutdown = Arc::new(Notify::new()); - let db_kind = if self.rpc_client_config.is_some() { - DatabaseKind::Replica - } else { - DatabaseKind::Primary - }; let scripted_backup = match self.db_config.snapshot_exec { Some(ref command) => { @@ -457,27 +452,6 @@ where // chose the wal backend let (make_wal_manager, registry_shutdown) = self.configure_wal_manager(&mut join_set)?; - let ns_config = NamespaceConfig { - db_kind, - base_path: self.path.clone(), - max_log_size: self.db_config.max_log_size, - max_log_duration: self.db_config.max_log_duration.map(Duration::from_secs_f32), - bottomless_replication: self.db_config.bottomless_replication.clone(), - extensions, - stats_sender: stats_sender.clone(), - max_response_size: self.db_config.max_response_size, - max_total_response_size: self.db_config.max_total_response_size, - checkpoint_interval: self.db_config.checkpoint_interval, - encryption_config: self.db_config.encryption_config.clone(), - max_concurrent_connections: Arc::new(Semaphore::new(self.max_concurrent_connections)), - scripted_backup, - max_concurrent_requests: self.db_config.max_concurrent_requests, - channel: channel.clone(), - uri: uri.clone(), - migration_scheduler: scheduler_sender.into(), - make_wal_manager, - }; - let (metastore_conn_maker, meta_store_wal_manager) = metastore_connection_maker(self.meta_store_config.bottomless.clone(), &self.path) .await?; @@ -490,15 +464,67 @@ where ) .await?; - let configurators = NamespaceConfigurators::default(); + let base_config = BaseNamespaceConfig { + base_path: self.path.clone(), + extensions, + stats_sender, + max_response_size: self.db_config.max_response_size, + max_total_response_size: self.db_config.max_total_response_size, + max_concurrent_connections: Arc::new(Semaphore::new(self.max_concurrent_connections)), + max_concurrent_requests: self.db_config.max_concurrent_requests, + }; + + let mut configurators = NamespaceConfigurators::default(); + + let db_kind = match channel.clone().zip(uri.clone()) { + // replica mode + Some((channel, uri)) => { + let replica_configurator = ReplicaConfigurator::new( + base_config, + channel, + uri, + make_wal_manager, + ); + configurators.with_replica(replica_configurator); + DatabaseKind::Replica + } + // primary mode + None => { + let primary_config = PrimaryExtraConfig { + max_log_size: self.db_config.max_log_size, + max_log_duration: self.db_config.max_log_duration.map(Duration::from_secs_f32), + bottomless_replication: self.db_config.bottomless_replication.clone(), + scripted_backup, + checkpoint_interval: self.db_config.checkpoint_interval, + }; + + let primary_configurator = PrimaryConfigurator::new( + base_config.clone(), + primary_config.clone(), + make_wal_manager.clone(), + ); + + let schema_configurator = SchemaConfigurator::new( + base_config.clone(), + primary_config, + make_wal_manager.clone(), + scheduler_sender.into(), + ); + + configurators.with_schema(schema_configurator); + configurators.with_primary(primary_configurator); + + DatabaseKind::Primary + }, + }; let namespace_store: NamespaceStore = NamespaceStore::new( db_kind.is_replica(), self.db_config.snapshot_at_shutdown, self.max_active_namespaces, - ns_config, meta_store, configurators, + db_kind, ) .await?; diff --git a/libsql-server/src/namespace/fork.rs b/libsql-server/src/namespace/configurator/fork.rs similarity index 77% rename from libsql-server/src/namespace/fork.rs rename to libsql-server/src/namespace/configurator/fork.rs index f25bf7a9a9..26a0b99b61 100644 --- a/libsql-server/src/namespace/fork.rs +++ b/libsql-server/src/namespace/configurator/fork.rs @@ -12,15 +12,71 @@ use tokio::io::{AsyncSeekExt, AsyncWriteExt}; use tokio::time::Duration; use tokio_stream::StreamExt; +use crate::database::Database; +use crate::namespace::meta_store::MetaStoreHandle; +use crate::namespace::{Namespace, NamespaceBottomlessDbId}; use crate::replication::primary::frame_stream::FrameStream; use crate::replication::{LogReadError, ReplicationLogger}; use crate::{BLOCKING_RT, LIBSQL_PAGE_SIZE}; -use super::meta_store::MetaStoreHandle; -use super::{NamespaceName, NamespaceStore, RestoreOption}; +use super::helpers::make_bottomless_options; +use super::{NamespaceName, NamespaceStore, PrimaryExtraConfig, RestoreOption}; type Result = crate::Result; +pub(super) async fn fork( + from_ns: &Namespace, + from_config: MetaStoreHandle, + to_ns: NamespaceName, + to_config: MetaStoreHandle, + timestamp: Option, + store: NamespaceStore, + primary_config: &PrimaryExtraConfig, + base_path: Arc, +) -> crate::Result { + let from_config = from_config.get(); + let bottomless_db_id = NamespaceBottomlessDbId::from_config(&from_config); + let restore_to = if let Some(timestamp) = timestamp { + if let Some(ref options) = primary_config.bottomless_replication { + Some(PointInTimeRestore { + timestamp, + replicator_options: make_bottomless_options( + options, + bottomless_db_id.clone(), + from_ns.name().clone(), + ), + }) + } else { + return Err(crate::Error::Fork(ForkError::BackupServiceNotConfigured)); + } + } else { + None + }; + + let logger = match &from_ns.db { + Database::Primary(db) => db.wal_wrapper.wrapper().logger(), + Database::Schema(db) => db.wal_wrapper.wrapper().logger(), + _ => { + return Err(crate::Error::Fork(ForkError::Internal(anyhow::Error::msg( + "Invalid source database type for fork", + )))); + } + }; + + let fork_task = ForkTask { + base_path, + to_namespace: to_ns.clone(), + logger, + restore_to, + to_config, + store, + }; + + let ns = fork_task.fork().await?; + + Ok(ns) +} + #[derive(Debug, thiserror::Error)] pub enum ForkError { #[error("internal error: {0}")] @@ -58,7 +114,7 @@ pub struct ForkTask { pub to_namespace: NamespaceName, pub to_config: MetaStoreHandle, pub restore_to: Option, - pub store: NamespaceStore, + pub store: NamespaceStore } pub struct PointInTimeRestore { diff --git a/libsql-server/src/namespace/configurator/helpers.rs b/libsql-server/src/namespace/configurator/helpers.rs new file mode 100644 index 0000000000..f43fa8a192 --- /dev/null +++ b/libsql-server/src/namespace/configurator/helpers.rs @@ -0,0 +1,451 @@ +use std::path::{Path, PathBuf}; +use std::sync::Weak; +use std::sync::{atomic::AtomicBool, Arc}; +use std::time::Duration; + +use anyhow::Context as _; +use bottomless::replicator::Options; +use bytes::Bytes; +use futures::Stream; +use libsql_sys::wal::Sqlite3WalManager; +use tokio::io::AsyncBufReadExt as _; +use tokio::sync::watch; +use tokio::task::JoinSet; +use tokio_util::io::StreamReader; +use enclose::enclose; + +use crate::connection::config::DatabaseConfig; +use crate::connection::connection_manager::InnerWalManager; +use crate::connection::libsql::{open_conn, MakeLibSqlConn}; +use crate::connection::{Connection as _, MakeConnection as _}; +use crate::error::LoadDumpError; +use crate::replication::{FrameNo, ReplicationLogger}; +use crate::stats::Stats; +use crate::namespace::{NamespaceBottomlessDbId, NamespaceBottomlessDbIdInit, NamespaceName, ResolveNamespacePathFn, RestoreOption}; +use crate::namespace::replication_wal::{make_replication_wal_wrapper, ReplicationWalWrapper}; +use crate::namespace::meta_store::MetaStoreHandle; +use crate::namespace::broadcasters::BroadcasterHandle; +use crate::database::{PrimaryConnection, PrimaryConnectionMaker}; +use crate::{StatsSender, BLOCKING_RT, DB_CREATE_TIMEOUT, DEFAULT_AUTO_CHECKPOINT}; + +use super::{BaseNamespaceConfig, PrimaryExtraConfig}; + +const WASM_TABLE_CREATE: &str = + "CREATE TABLE libsql_wasm_func_table (name text PRIMARY KEY, body text) WITHOUT ROWID;"; + +#[tracing::instrument(skip_all)] +pub(super) async fn make_primary_connection_maker( + primary_config: &PrimaryExtraConfig, + base_config: &BaseNamespaceConfig, + meta_store_handle: &MetaStoreHandle, + db_path: &Path, + name: &NamespaceName, + restore_option: RestoreOption, + block_writes: Arc, + join_set: &mut JoinSet>, + resolve_attach_path: ResolveNamespacePathFn, + broadcaster: BroadcasterHandle, + make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, +) -> crate::Result<(PrimaryConnectionMaker, ReplicationWalWrapper, Arc)> { + let db_config = meta_store_handle.get(); + let bottomless_db_id = NamespaceBottomlessDbId::from_config(&db_config); + // FIXME: figure how to to it per-db + let mut is_dirty = { + let sentinel_path = db_path.join(".sentinel"); + if sentinel_path.try_exists()? { + true + } else { + tokio::fs::File::create(&sentinel_path).await?; + false + } + }; + + // FIXME: due to a bug in logger::checkpoint_db we call regular checkpointing code + // instead of our virtual WAL one. It's a bit tangled to fix right now, because + // we need WAL context for checkpointing, and WAL context needs the ReplicationLogger... + // So instead we checkpoint early, *before* bottomless gets initialized. That way + // we're sure bottomless won't try to back up any existing WAL frames and will instead + // treat the existing db file as the source of truth. + + let bottomless_replicator = match primary_config.bottomless_replication { + Some(ref options) => { + tracing::debug!("Checkpointing before initializing bottomless"); + crate::replication::primary::logger::checkpoint_db(&db_path.join("data"))?; + tracing::debug!("Checkpointed before initializing bottomless"); + let options = make_bottomless_options(options, bottomless_db_id, name.clone()); + let (replicator, did_recover) = + init_bottomless_replicator(db_path.join("data"), options, &restore_option) + .await?; + tracing::debug!("Completed init of bottomless replicator"); + is_dirty |= did_recover; + Some(replicator) + } + None => None, + }; + + tracing::debug!("Checking fresh db"); + let is_fresh_db = check_fresh_db(&db_path)?; + // switch frame-count checkpoint to time-based one + let auto_checkpoint = if primary_config.checkpoint_interval.is_some() { + 0 + } else { + DEFAULT_AUTO_CHECKPOINT + }; + + let logger = Arc::new(ReplicationLogger::open( + &db_path, + primary_config.max_log_size, + primary_config.max_log_duration, + is_dirty, + auto_checkpoint, + primary_config.scripted_backup.clone(), + name.clone(), + None, + )?); + + tracing::debug!("sending stats"); + + let stats = make_stats( + &db_path, + join_set, + meta_store_handle.clone(), + base_config.stats_sender.clone(), + name.clone(), + logger.new_frame_notifier.subscribe(), + ) + .await?; + + tracing::debug!("Making replication wal wrapper"); + let wal_wrapper = make_replication_wal_wrapper(bottomless_replicator, logger.clone()); + + tracing::debug!("Opening libsql connection"); + + let connection_maker = MakeLibSqlConn::new( + db_path.to_path_buf(), + wal_wrapper.clone(), + stats.clone(), + broadcaster, + meta_store_handle.clone(), + base_config.extensions.clone(), + base_config.max_response_size, + base_config.max_total_response_size, + auto_checkpoint, + logger.new_frame_notifier.subscribe(), + None, + block_writes, + resolve_attach_path, + make_wal_manager.clone(), + ) + .await? + .throttled( + base_config.max_concurrent_connections.clone(), + Some(DB_CREATE_TIMEOUT), + base_config.max_total_response_size, + base_config.max_concurrent_requests, + ); + + tracing::debug!("Completed opening libsql connection"); + + // this must happen after we create the connection maker. The connection maker old on a + // connection to ensure that no other connection is closing while we try to open the dump. + // that would cause a SQLITE_LOCKED error. + match restore_option { + RestoreOption::Dump(_) if !is_fresh_db => { + Err(LoadDumpError::LoadDumpExistingDb)?; + } + RestoreOption::Dump(dump) => { + let conn = connection_maker.create().await?; + tracing::debug!("Loading dump"); + load_dump(dump, conn).await?; + tracing::debug!("Done loading dump"); + } + _ => { /* other cases were already handled when creating bottomless */ } + } + + join_set.spawn(run_periodic_compactions(logger.clone())); + + tracing::debug!("Done making primary connection"); + + Ok((connection_maker, wal_wrapper, stats)) +} + +pub(super) fn make_bottomless_options( + options: &Options, + namespace_db_id: NamespaceBottomlessDbId, + name: NamespaceName, +) -> Options { + let mut options = options.clone(); + let mut db_id = match namespace_db_id { + NamespaceBottomlessDbId::Namespace(id) => id, + // FIXME(marin): I don't like that, if bottomless is enabled, proper config must be passed. + NamespaceBottomlessDbId::NotProvided => options.db_id.unwrap_or_default(), + }; + + db_id = format!("ns-{db_id}:{name}"); + options.db_id = Some(db_id); + options +} + +async fn init_bottomless_replicator( + path: impl AsRef, + options: bottomless::replicator::Options, + restore_option: &RestoreOption, +) -> anyhow::Result<(bottomless::replicator::Replicator, bool)> { + tracing::debug!("Initializing bottomless replication"); + let path = path + .as_ref() + .to_str() + .ok_or_else(|| anyhow::anyhow!("Invalid db path"))? + .to_owned(); + let mut replicator = bottomless::replicator::Replicator::with_options(path, options).await?; + + let (generation, timestamp) = match restore_option { + RestoreOption::Latest | RestoreOption::Dump(_) => (None, None), + RestoreOption::Generation(generation) => (Some(*generation), None), + RestoreOption::PointInTime(timestamp) => (None, Some(*timestamp)), + }; + + let (action, did_recover) = replicator.restore(generation, timestamp).await?; + match action { + bottomless::replicator::RestoreAction::SnapshotMainDbFile => { + replicator.new_generation().await; + if let Some(_handle) = replicator.snapshot_main_db_file(true).await? { + tracing::trace!("got snapshot handle after restore with generation upgrade"); + } + // Restoration process only leaves the local WAL file if it was + // detected to be newer than its remote counterpart. + replicator.maybe_replicate_wal().await? + } + bottomless::replicator::RestoreAction::ReuseGeneration(gen) => { + replicator.set_generation(gen); + } + } + + Ok((replicator, did_recover)) +} + +async fn run_periodic_compactions(logger: Arc) -> anyhow::Result<()> { + // calling `ReplicationLogger::maybe_compact()` is cheap if the compaction does not actually + // take place, so we can afford to poll it very often for simplicity + let mut interval = tokio::time::interval(tokio::time::Duration::from_millis(1000)); + interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + + loop { + interval.tick().await; + let handle = BLOCKING_RT.spawn_blocking(enclose! {(logger) move || { + logger.maybe_compact() + }}); + handle + .await + .expect("Compaction task crashed") + .context("Compaction failed")?; + } +} + +async fn load_dump(dump: S, conn: PrimaryConnection) -> crate::Result<(), LoadDumpError> +where + S: Stream> + Unpin, +{ + let mut reader = tokio::io::BufReader::new(StreamReader::new(dump)); + let mut curr = String::new(); + let mut line = String::new(); + let mut skipped_wasm_table = false; + let mut n_stmt = 0; + let mut line_id = 0; + + while let Ok(n) = reader.read_line(&mut curr).await { + line_id += 1; + if n == 0 { + break; + } + let trimmed = curr.trim(); + if trimmed.is_empty() || trimmed.starts_with("--") { + curr.clear(); + continue; + } + // FIXME: it's well known bug that comment ending with semicolon will be handled incorrectly by currend dump processing code + let statement_end = trimmed.ends_with(';'); + + // we want to concat original(non-trimmed) lines as trimming will join all them in one + // single-line statement which is incorrect if comments in the end are present + line.push_str(&curr); + curr.clear(); + + // This is a hack to ignore the libsql_wasm_func_table table because it is already created + // by the system. + if !skipped_wasm_table && line.trim() == WASM_TABLE_CREATE { + skipped_wasm_table = true; + line.clear(); + continue; + } + + if statement_end { + n_stmt += 1; + // dump must be performd within a txn + if n_stmt > 2 && conn.is_autocommit().await.unwrap() { + return Err(LoadDumpError::NoTxn); + } + + line = tokio::task::spawn_blocking({ + let conn = conn.clone(); + move || -> crate::Result { + conn.with_raw(|conn| conn.execute(&line, ())).map_err(|e| { + LoadDumpError::Internal(format!("line: {}, error: {}", line_id, e)) + })?; + Ok(line) + } + }) + .await??; + line.clear(); + } else { + line.push(' '); + } + } + tracing::debug!("loaded {} lines from dump", line_id); + + if !conn.is_autocommit().await.unwrap() { + tokio::task::spawn_blocking({ + let conn = conn.clone(); + move || -> crate::Result<(), LoadDumpError> { + conn.with_raw(|conn| conn.execute("rollback", ()))?; + Ok(()) + } + }) + .await??; + return Err(LoadDumpError::NoCommit); + } + + Ok(()) +} + +fn check_fresh_db(path: &Path) -> crate::Result { + let is_fresh = !path.join("wallog").try_exists()?; + Ok(is_fresh) +} + +pub(super) async fn make_stats( + db_path: &Path, + join_set: &mut JoinSet>, + meta_store_handle: MetaStoreHandle, + stats_sender: StatsSender, + name: NamespaceName, + mut current_frame_no: watch::Receiver>, +) -> anyhow::Result> { + tracing::debug!("creating stats type"); + let stats = Stats::new(name.clone(), db_path, join_set).await?; + + // the storage monitor is optional, so we ignore the error here. + tracing::debug!("stats created, sending stats"); + let _ = stats_sender + .send((name.clone(), meta_store_handle, Arc::downgrade(&stats))) + .await; + + join_set.spawn({ + let stats = stats.clone(); + // initialize the current_frame_no value + current_frame_no + .borrow_and_update() + .map(|fno| stats.set_current_frame_no(fno)); + async move { + while current_frame_no.changed().await.is_ok() { + current_frame_no + .borrow_and_update() + .map(|fno| stats.set_current_frame_no(fno)); + } + Ok(()) + } + }); + + join_set.spawn(run_storage_monitor( + db_path.into(), + Arc::downgrade(&stats), + )); + + tracing::debug!("done sending stats, and creating bg tasks"); + + Ok(stats) +} + +// Periodically check the storage used by the database and save it in the Stats structure. +// TODO: Once we have a separate fiber that does WAL checkpoints, running this routine +// right after checkpointing is exactly where it should be done. +async fn run_storage_monitor( + db_path: PathBuf, + stats: Weak, +) -> anyhow::Result<()> { + // on initialization, the database file doesn't exist yet, so we wait a bit for it to be + // created + tokio::time::sleep(Duration::from_secs(1)).await; + + let duration = tokio::time::Duration::from_secs(60); + let db_path: Arc = db_path.into(); + loop { + let db_path = db_path.clone(); + let Some(stats) = stats.upgrade() else { + return Ok(()); + }; + + let _ = tokio::task::spawn_blocking(move || { + // because closing the last connection interferes with opening a new one, we lazily + // initialize a connection here, and keep it alive for the entirety of the program. If we + // fail to open it, we wait for `duration` and try again later. + match open_conn(&db_path, Sqlite3WalManager::new(), Some(rusqlite::OpenFlags::SQLITE_OPEN_READ_ONLY), None) { + Ok(mut conn) => { + if let Ok(tx) = conn.transaction() { + let page_count = tx.query_row("pragma page_count;", [], |row| { row.get::(0) }); + let freelist_count = tx.query_row("pragma freelist_count;", [], |row| { row.get::(0) }); + if let (Ok(page_count), Ok(freelist_count)) = (page_count, freelist_count) { + let storage_bytes_used = (page_count - freelist_count) * 4096; + stats.set_storage_bytes_used(storage_bytes_used); + } + } + }, + Err(e) => { + tracing::warn!("failed to open connection for storager monitor: {e}, trying again in {duration:?}"); + }, + } + }).await; + + tokio::time::sleep(duration).await; + } +} + +pub(super) async fn cleanup_primary( + base: &BaseNamespaceConfig, + primary_config: &PrimaryExtraConfig, + namespace: &NamespaceName, + db_config: &DatabaseConfig, + prune_all: bool, + bottomless_db_id_init: NamespaceBottomlessDbIdInit, +) -> crate::Result<()> { + let ns_path = base.base_path.join("dbs").join(namespace.as_str()); + if let Some(ref options) = primary_config.bottomless_replication { + let bottomless_db_id = match bottomless_db_id_init { + NamespaceBottomlessDbIdInit::Provided(db_id) => db_id, + NamespaceBottomlessDbIdInit::FetchFromConfig => { + NamespaceBottomlessDbId::from_config(db_config) + } + }; + let options = make_bottomless_options(options, bottomless_db_id, namespace.clone()); + let replicator = bottomless::replicator::Replicator::with_options( + ns_path.join("data").to_str().unwrap(), + options, + ) + .await?; + if prune_all { + let delete_all = replicator.delete_all(None).await?; + // perform hard deletion in the background + tokio::spawn(delete_all.commit()); + } else { + // for soft delete make sure that local db is fully backed up + replicator.savepoint().confirmed().await?; + } + } + + if ns_path.try_exists()? { + tracing::debug!("removing database directory: {}", ns_path.display()); + tokio::fs::remove_dir_all(ns_path).await?; + } + + Ok(()) +} diff --git a/libsql-server/src/namespace/configurator/mod.rs b/libsql-server/src/namespace/configurator/mod.rs index a240c3e410..e5db335ff6 100644 --- a/libsql-server/src/namespace/configurator/mod.rs +++ b/libsql-server/src/namespace/configurator/mod.rs @@ -1,22 +1,51 @@ +use std::path::{Path, PathBuf}; use std::pin::Pin; +use std::sync::Arc; +use std::time::Duration; +use chrono::NaiveDateTime; use futures::Future; +use tokio::sync::Semaphore; + +use crate::connection::config::DatabaseConfig; +use crate::replication::script_backup_manager::ScriptBackupManager; +use crate::StatsSender; use super::broadcasters::BroadcasterHandle; use super::meta_store::MetaStoreHandle; -use super::{ - NamespaceConfig, NamespaceName, NamespaceStore, ResetCb, ResolveNamespacePathFn, RestoreOption, -}; +use super::{Namespace, NamespaceBottomlessDbIdInit, NamespaceName, NamespaceStore, ResetCb, ResolveNamespacePathFn, RestoreOption}; +mod helpers; mod primary; mod replica; mod schema; +pub mod fork; pub use primary::PrimaryConfigurator; pub use replica::ReplicaConfigurator; pub use schema::SchemaConfigurator; -type DynConfigurator = dyn ConfigureNamespace + Send + Sync + 'static; +#[derive(Clone, Debug)] +pub struct BaseNamespaceConfig { + pub(crate) base_path: Arc, + pub(crate) extensions: Arc<[PathBuf]>, + pub(crate) stats_sender: StatsSender, + pub(crate) max_response_size: u64, + pub(crate) max_total_response_size: u64, + pub(crate) max_concurrent_connections: Arc, + pub(crate) max_concurrent_requests: u64, +} + +#[derive(Clone)] +pub struct PrimaryExtraConfig { + pub(crate) max_log_size: u64, + pub(crate) max_log_duration: Option, + pub(crate) bottomless_replication: Option, + pub(crate) scripted_backup: Option, + pub(crate) checkpoint_interval: Option, +} + +pub type DynConfigurator = dyn ConfigureNamespace + Send + Sync + 'static; pub(crate) struct NamespaceConfigurators { replica_configurator: Option>, @@ -27,9 +56,6 @@ pub(crate) struct NamespaceConfigurators { impl Default for NamespaceConfigurators { fn default() -> Self { Self::empty() - .with_primary(PrimaryConfigurator) - .with_replica(ReplicaConfigurator) - .with_schema(SchemaConfigurator) } } @@ -42,17 +68,17 @@ impl NamespaceConfigurators { } } - pub fn with_primary(mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> Self { + pub fn with_primary(&mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> &mut Self { self.primary_configurator = Some(Box::new(c)); self } - pub fn with_replica(mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> Self { + pub fn with_replica(&mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> &mut Self { self.replica_configurator = Some(Box::new(c)); self } - pub fn with_schema(mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> Self { + pub fn with_schema(&mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> &mut Self { self.schema_configurator = Some(Box::new(c)); self } @@ -73,7 +99,6 @@ impl NamespaceConfigurators { pub trait ConfigureNamespace { fn setup<'a>( &'a self, - ns_config: &'a NamespaceConfig, db_config: MetaStoreHandle, restore_option: RestoreOption, name: &'a NamespaceName, @@ -81,5 +106,23 @@ pub trait ConfigureNamespace { resolve_attach_path: ResolveNamespacePathFn, store: NamespaceStore, broadcaster: BroadcasterHandle, - ) -> Pin> + Send + 'a>>; + ) -> Pin> + Send + 'a>>; + + fn cleanup<'a>( + &'a self, + namespace: &'a NamespaceName, + db_config: &'a DatabaseConfig, + prune_all: bool, + bottomless_db_id_init: NamespaceBottomlessDbIdInit, + ) -> Pin> + Send + 'a>>; + + fn fork<'a>( + &'a self, + from_ns: &'a Namespace, + from_config: MetaStoreHandle, + to_ns: NamespaceName, + to_config: MetaStoreHandle, + timestamp: Option, + store: NamespaceStore, + ) -> Pin> + Send + 'a>>; } diff --git a/libsql-server/src/namespace/configurator/primary.rs b/libsql-server/src/namespace/configurator/primary.rs index f28d288a97..4351f6a3ac 100644 --- a/libsql-server/src/namespace/configurator/primary.rs +++ b/libsql-server/src/namespace/configurator/primary.rs @@ -4,22 +4,117 @@ use std::{path::Path, pin::Pin, sync::Arc}; use futures::prelude::Future; use tokio::task::JoinSet; +use crate::connection::config::DatabaseConfig; +use crate::connection::connection_manager::InnerWalManager; use crate::connection::MakeConnection; use crate::database::{Database, PrimaryDatabase}; -use crate::namespace::{Namespace, NamespaceConfig, NamespaceName, NamespaceStore, ResetCb, ResolveNamespacePathFn, RestoreOption}; -use crate::namespace::meta_store::MetaStoreHandle; use crate::namespace::broadcasters::BroadcasterHandle; +use crate::namespace::configurator::helpers::make_primary_connection_maker; +use crate::namespace::meta_store::MetaStoreHandle; +use crate::namespace::{ + Namespace, NamespaceBottomlessDbIdInit, NamespaceName, NamespaceStore, + ResetCb, ResolveNamespacePathFn, RestoreOption, +}; use crate::run_periodic_checkpoint; use crate::schema::{has_pending_migration_task, setup_migration_table}; -use super::ConfigureNamespace; +use super::helpers::cleanup_primary; +use super::{BaseNamespaceConfig, ConfigureNamespace, PrimaryExtraConfig}; + +pub struct PrimaryConfigurator { + base: BaseNamespaceConfig, + primary_config: PrimaryExtraConfig, + make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, +} + +impl PrimaryConfigurator { + pub fn new( + base: BaseNamespaceConfig, + primary_config: PrimaryExtraConfig, + make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, + ) -> Self { + Self { + base, + primary_config, + make_wal_manager, + } + } + + #[tracing::instrument(skip_all, fields(namespace))] + async fn try_new_primary( + &self, + namespace: NamespaceName, + meta_store_handle: MetaStoreHandle, + restore_option: RestoreOption, + resolve_attach_path: ResolveNamespacePathFn, + db_path: Arc, + broadcaster: BroadcasterHandle, + ) -> crate::Result { + let mut join_set = JoinSet::new(); + + tokio::fs::create_dir_all(&db_path).await?; + + let block_writes = Arc::new(AtomicBool::new(false)); + let (connection_maker, wal_wrapper, stats) = make_primary_connection_maker( + &self.primary_config, + &self.base, + &meta_store_handle, + &db_path, + &namespace, + restore_option, + block_writes.clone(), + &mut join_set, + resolve_attach_path, + broadcaster, + self.make_wal_manager.clone(), + ) + .await?; + let connection_maker = Arc::new(connection_maker); + + if meta_store_handle.get().shared_schema_name.is_some() { + let block_writes = block_writes.clone(); + let conn = connection_maker.create().await?; + tokio::task::spawn_blocking(move || { + conn.with_raw(|conn| -> crate::Result<()> { + setup_migration_table(conn)?; + if has_pending_migration_task(conn)? { + block_writes.store(true, Ordering::SeqCst); + } + Ok(()) + }) + }) + .await + .unwrap()?; + } + + if let Some(checkpoint_interval) = self.primary_config.checkpoint_interval { + join_set.spawn(run_periodic_checkpoint( + connection_maker.clone(), + checkpoint_interval, + namespace.clone(), + )); + } + + tracing::debug!("Done making new primary"); -pub struct PrimaryConfigurator; + Ok(Namespace { + tasks: join_set, + db: Database::Primary(PrimaryDatabase { + wal_wrapper, + connection_maker, + block_writes, + }), + name: namespace, + stats, + db_config_store: meta_store_handle, + path: db_path.into(), + }) + } +} impl ConfigureNamespace for PrimaryConfigurator { fn setup<'a>( &'a self, - config: &'a NamespaceConfig, meta_store_handle: MetaStoreHandle, restore_option: RestoreOption, name: &'a NamespaceName, @@ -27,102 +122,74 @@ impl ConfigureNamespace for PrimaryConfigurator { resolve_attach_path: ResolveNamespacePathFn, _store: NamespaceStore, broadcaster: BroadcasterHandle, - ) -> Pin> + Send + 'a>> - { + ) -> Pin> + Send + 'a>> { Box::pin(async move { - let db_path: Arc = config.base_path.join("dbs").join(name.as_str()).into(); + let db_path: Arc = self.base.base_path.join("dbs").join(name.as_str()).into(); let fresh_namespace = !db_path.try_exists()?; // FIXME: make that truly atomic. explore the idea of using temp directories, and it's implications - match try_new_primary( - config, - name.clone(), - meta_store_handle, - restore_option, - resolve_attach_path, - db_path.clone(), - broadcaster, - ) + match self + .try_new_primary( + name.clone(), + meta_store_handle, + restore_option, + resolve_attach_path, + db_path.clone(), + broadcaster, + ) .await - { - Ok(this) => Ok(this), - Err(e) if fresh_namespace => { - tracing::error!("an error occured while deleting creating namespace, cleaning..."); - if let Err(e) = tokio::fs::remove_dir_all(&db_path).await { - tracing::error!("failed to remove dirty namespace directory: {e}") - } - Err(e) + { + Ok(this) => Ok(this), + Err(e) if fresh_namespace => { + tracing::error!( + "an error occured while deleting creating namespace, cleaning..." + ); + if let Err(e) = tokio::fs::remove_dir_all(&db_path).await { + tracing::error!("failed to remove dirty namespace directory: {e}") } - Err(e) => Err(e), + Err(e) } + Err(e) => Err(e), + } }) } -} -#[tracing::instrument(skip_all, fields(namespace))] -async fn try_new_primary( - ns_config: &NamespaceConfig, - namespace: NamespaceName, - meta_store_handle: MetaStoreHandle, - restore_option: RestoreOption, - resolve_attach_path: ResolveNamespacePathFn, - db_path: Arc, - broadcaster: BroadcasterHandle, -) -> crate::Result { - let mut join_set = JoinSet::new(); - - tokio::fs::create_dir_all(&db_path).await?; - - let block_writes = Arc::new(AtomicBool::new(false)); - let (connection_maker, wal_wrapper, stats) = Namespace::make_primary_connection_maker( - ns_config, - &meta_store_handle, - &db_path, - &namespace, - restore_option, - block_writes.clone(), - &mut join_set, - resolve_attach_path, - broadcaster, - ) - .await?; - let connection_maker = Arc::new(connection_maker); - - if meta_store_handle.get().shared_schema_name.is_some() { - let block_writes = block_writes.clone(); - let conn = connection_maker.create().await?; - tokio::task::spawn_blocking(move || { - conn.with_raw(|conn| -> crate::Result<()> { - setup_migration_table(conn)?; - if has_pending_migration_task(conn)? { - block_writes.store(true, Ordering::SeqCst); - } - Ok(()) - }) + fn cleanup<'a>( + &'a self, + namespace: &'a NamespaceName, + db_config: &'a DatabaseConfig, + prune_all: bool, + bottomless_db_id_init: NamespaceBottomlessDbIdInit, + ) -> Pin> + Send + 'a>> { + Box::pin(async move { + cleanup_primary( + &self.base, + &self.primary_config, + namespace, + db_config, + prune_all, + bottomless_db_id_init, + ).await }) - .await - .unwrap()?; - } - - if let Some(checkpoint_interval) = ns_config.checkpoint_interval { - join_set.spawn(run_periodic_checkpoint( - connection_maker.clone(), - checkpoint_interval, - namespace.clone(), - )); } - tracing::debug!("Done making new primary"); - - Ok(Namespace { - tasks: join_set, - db: Database::Primary(PrimaryDatabase { - wal_wrapper, - connection_maker, - block_writes, - }), - name: namespace, - stats, - db_config_store: meta_store_handle, - path: db_path.into(), - }) + fn fork<'a>( + &'a self, + from_ns: &'a Namespace, + from_config: MetaStoreHandle, + to_ns: NamespaceName, + to_config: MetaStoreHandle, + timestamp: Option, + store: NamespaceStore, + ) -> Pin> + Send + 'a>> { + Box::pin(super::fork::fork( + from_ns, + from_config, + to_ns, + to_config, + timestamp, + store, + &self.primary_config, + self.base.base_path.clone())) + } } + diff --git a/libsql-server/src/namespace/configurator/replica.rs b/libsql-server/src/namespace/configurator/replica.rs index 4d3ca1dadf..61dd48b0bf 100644 --- a/libsql-server/src/namespace/configurator/replica.rs +++ b/libsql-server/src/namespace/configurator/replica.rs @@ -2,29 +2,51 @@ use std::pin::Pin; use std::sync::Arc; use futures::Future; +use hyper::Uri; use libsql_replication::rpc::replication::replication_log_client::ReplicationLogClient; use tokio::task::JoinSet; +use tonic::transport::Channel; +use crate::connection::config::DatabaseConfig; +use crate::connection::connection_manager::InnerWalManager; use crate::connection::write_proxy::MakeWriteProxyConn; use crate::connection::MakeConnection; use crate::database::{Database, ReplicaDatabase}; use crate::namespace::broadcasters::BroadcasterHandle; +use crate::namespace::configurator::helpers::make_stats; use crate::namespace::meta_store::MetaStoreHandle; -use crate::namespace::{Namespace, RestoreOption}; -use crate::namespace::{ - make_stats, NamespaceConfig, NamespaceName, NamespaceStore, ResetCb, ResetOp, - ResolveNamespacePathFn, -}; +use crate::namespace::{Namespace, NamespaceBottomlessDbIdInit, RestoreOption}; +use crate::namespace::{NamespaceName, NamespaceStore, ResetCb, ResetOp, ResolveNamespacePathFn}; use crate::{DB_CREATE_TIMEOUT, DEFAULT_AUTO_CHECKPOINT}; -use super::ConfigureNamespace; +use super::{BaseNamespaceConfig, ConfigureNamespace}; -pub struct ReplicaConfigurator; +pub struct ReplicaConfigurator { + base: BaseNamespaceConfig, + channel: Channel, + uri: Uri, + make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, +} + +impl ReplicaConfigurator { + pub fn new( + base: BaseNamespaceConfig, + channel: Channel, + uri: Uri, + make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, + ) -> Self { + Self { + base, + channel, + uri, + make_wal_manager, + } + } +} impl ConfigureNamespace for ReplicaConfigurator { fn setup<'a>( &'a self, - config: &'a NamespaceConfig, meta_store_handle: MetaStoreHandle, restore_option: RestoreOption, name: &'a NamespaceName, @@ -32,13 +54,12 @@ impl ConfigureNamespace for ReplicaConfigurator { resolve_attach_path: ResolveNamespacePathFn, store: NamespaceStore, broadcaster: BroadcasterHandle, - ) -> Pin> + Send + 'a>> - { + ) -> Pin> + Send + 'a>> { Box::pin(async move { tracing::debug!("creating replica namespace"); - let db_path = config.base_path.join("dbs").join(name.as_str()); - let channel = config.channel.clone().expect("bad replica config"); - let uri = config.uri.clone().expect("bad replica config"); + let db_path = self.base.base_path.join("dbs").join(name.as_str()); + let channel = self.channel.clone(); + let uri = self.uri.clone(); let rpc_client = ReplicationLogClient::with_origin(channel.clone(), uri.clone()); let client = crate::replication::replicator_client::Client::new( @@ -48,45 +69,46 @@ impl ConfigureNamespace for ReplicaConfigurator { meta_store_handle.clone(), store.clone(), ) - .await?; + .await?; let applied_frame_no_receiver = client.current_frame_no_notifier.subscribe(); let mut replicator = libsql_replication::replicator::Replicator::new( client, db_path.join("data"), DEFAULT_AUTO_CHECKPOINT, - config.encryption_config.clone(), + None, ) - .await?; + .await?; tracing::debug!("try perform handshake"); // force a handshake now, to retrieve the primary's current replication index match replicator.try_perform_handshake().await { Err(libsql_replication::replicator::Error::Meta( - libsql_replication::meta::Error::LogIncompatible, + libsql_replication::meta::Error::LogIncompatible, )) => { tracing::error!( "trying to replicate incompatible logs, reseting replica and nuking db dir" ); std::fs::remove_dir_all(&db_path).unwrap(); - return self.setup( - config, - meta_store_handle, - restore_option, - name, - reset, - resolve_attach_path, - store, - broadcaster, - ) + return self + .setup( + meta_store_handle, + restore_option, + name, + reset, + resolve_attach_path, + store, + broadcaster, + ) .await; - } + } Err(e) => Err(e)?, Ok(_) => (), } tracing::debug!("done performing handshake"); - let primary_current_replicatio_index = replicator.client_mut().primary_replication_index; + let primary_current_replicatio_index = + replicator.client_mut().primary_replication_index; let mut join_set = JoinSet::new(); let namespace = name.clone(); @@ -144,36 +166,35 @@ impl ConfigureNamespace for ReplicaConfigurator { &db_path, &mut join_set, meta_store_handle.clone(), - config.stats_sender.clone(), + self.base.stats_sender.clone(), name.clone(), applied_frame_no_receiver.clone(), - config.encryption_config.clone(), ) - .await?; + .await?; let connection_maker = MakeWriteProxyConn::new( db_path.clone(), - config.extensions.clone(), + self.base.extensions.clone(), channel.clone(), uri.clone(), stats.clone(), broadcaster, meta_store_handle.clone(), applied_frame_no_receiver, - config.max_response_size, - config.max_total_response_size, + self.base.max_response_size, + self.base.max_total_response_size, primary_current_replicatio_index, - config.encryption_config.clone(), + None, resolve_attach_path, - config.make_wal_manager.clone(), + self.make_wal_manager.clone(), ) - .await? - .throttled( - config.max_concurrent_connections.clone(), - Some(DB_CREATE_TIMEOUT), - config.max_total_response_size, - config.max_concurrent_requests, - ); + .await? + .throttled( + self.base.max_concurrent_connections.clone(), + Some(DB_CREATE_TIMEOUT), + self.base.max_total_response_size, + self.base.max_concurrent_requests, + ); Ok(Namespace { tasks: join_set, @@ -187,4 +208,35 @@ impl ConfigureNamespace for ReplicaConfigurator { }) }) } + + fn cleanup<'a>( + &'a self, + namespace: &'a NamespaceName, + _db_config: &DatabaseConfig, + _prune_all: bool, + _bottomless_db_id_init: NamespaceBottomlessDbIdInit, + ) -> Pin> + Send + 'a>> { + Box::pin(async move { + let ns_path = self.base.base_path.join("dbs").join(namespace.as_str()); + if ns_path.try_exists()? { + tracing::debug!("removing database directory: {}", ns_path.display()); + tokio::fs::remove_dir_all(ns_path).await?; + } + Ok(()) + }) + } + + fn fork<'a>( + &'a self, + _from_ns: &'a Namespace, + _from_config: MetaStoreHandle, + _to_ns: NamespaceName, + _to_config: MetaStoreHandle, + _timestamp: Option, + _store: NamespaceStore, + ) -> Pin> + Send + 'a>> { + Box::pin(std::future::ready(Err(crate::Error::Fork( + super::fork::ForkError::ForkReplica, + )))) + } } diff --git a/libsql-server/src/namespace/configurator/schema.rs b/libsql-server/src/namespace/configurator/schema.rs index 864b75239f..e55c706fec 100644 --- a/libsql-server/src/namespace/configurator/schema.rs +++ b/libsql-server/src/namespace/configurator/schema.rs @@ -3,22 +3,36 @@ use std::sync::{atomic::AtomicBool, Arc}; use futures::prelude::Future; use tokio::task::JoinSet; +use crate::connection::config::DatabaseConfig; +use crate::connection::connection_manager::InnerWalManager; use crate::database::{Database, SchemaDatabase}; use crate::namespace::meta_store::MetaStoreHandle; use crate::namespace::{ - Namespace, NamespaceConfig, NamespaceName, NamespaceStore, + Namespace, NamespaceName, NamespaceStore, ResetCb, ResolveNamespacePathFn, RestoreOption, }; use crate::namespace::broadcasters::BroadcasterHandle; +use crate::schema::SchedulerHandle; -use super::ConfigureNamespace; +use super::helpers::{cleanup_primary, make_primary_connection_maker}; +use super::{BaseNamespaceConfig, ConfigureNamespace, PrimaryExtraConfig}; -pub struct SchemaConfigurator; +pub struct SchemaConfigurator { + base: BaseNamespaceConfig, + primary_config: PrimaryExtraConfig, + make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, + migration_scheduler: SchedulerHandle, +} + +impl SchemaConfigurator { + pub fn new(base: BaseNamespaceConfig, primary_config: PrimaryExtraConfig, make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, migration_scheduler: SchedulerHandle) -> Self { + Self { base, primary_config, make_wal_manager, migration_scheduler } + } +} impl ConfigureNamespace for SchemaConfigurator { fn setup<'a>( &'a self, - ns_config: &'a NamespaceConfig, db_config: MetaStoreHandle, restore_option: RestoreOption, name: &'a NamespaceName, @@ -29,12 +43,13 @@ impl ConfigureNamespace for SchemaConfigurator { ) -> std::pin::Pin> + Send + 'a>> { Box::pin(async move { let mut join_set = JoinSet::new(); - let db_path = ns_config.base_path.join("dbs").join(name.as_str()); + let db_path = self.base.base_path.join("dbs").join(name.as_str()); tokio::fs::create_dir_all(&db_path).await?; - let (connection_maker, wal_manager, stats) = Namespace::make_primary_connection_maker( - ns_config, + let (connection_maker, wal_manager, stats) = make_primary_connection_maker( + &self.primary_config, + &self.base, &db_config, &db_path, &name, @@ -43,12 +58,13 @@ impl ConfigureNamespace for SchemaConfigurator { &mut join_set, resolve_attach_path, broadcaster, + self.make_wal_manager.clone() ) .await?; Ok(Namespace { db: Database::Schema(SchemaDatabase::new( - ns_config.migration_scheduler.clone(), + self.migration_scheduler.clone(), name.clone(), connection_maker, wal_manager, @@ -62,4 +78,43 @@ impl ConfigureNamespace for SchemaConfigurator { }) }) } + + fn cleanup<'a>( + &'a self, + namespace: &'a NamespaceName, + db_config: &'a DatabaseConfig, + prune_all: bool, + bottomless_db_id_init: crate::namespace::NamespaceBottomlessDbIdInit, + ) -> std::pin::Pin> + Send + 'a>> { + Box::pin(async move { + cleanup_primary( + &self.base, + &self.primary_config, + namespace, + db_config, + prune_all, + bottomless_db_id_init, + ).await + }) + } + + fn fork<'a>( + &'a self, + from_ns: &'a Namespace, + from_config: MetaStoreHandle, + to_ns: NamespaceName, + to_config: MetaStoreHandle, + timestamp: Option, + store: NamespaceStore, + ) -> std::pin::Pin> + Send + 'a>> { + Box::pin(super::fork::fork( + from_ns, + from_config, + to_ns, + to_config, + timestamp, + store, + &self.primary_config, + self.base.base_path.clone())) + } } diff --git a/libsql-server/src/namespace/mod.rs b/libsql-server/src/namespace/mod.rs index 5ccda74c54..7cfa6b351c 100644 --- a/libsql-server/src/namespace/mod.rs +++ b/libsql-server/src/namespace/mod.rs @@ -1,52 +1,24 @@ -use std::path::{Path, PathBuf}; -use std::sync::atomic::AtomicBool; -use std::sync::{Arc, Weak}; +use std::path::Path; +use std::sync::Arc; -use anyhow::{Context as _, Error}; -use bottomless::replicator::Options; -use broadcasters::BroadcasterHandle; +use anyhow::Context as _; use bytes::Bytes; use chrono::NaiveDateTime; -use enclose::enclose; use futures_core::{Future, Stream}; -use hyper::Uri; -use libsql_sys::wal::Sqlite3WalManager; -use libsql_sys::EncryptionConfig; -use tokio::io::AsyncBufReadExt; -use tokio::sync::{watch, Semaphore}; use tokio::task::JoinSet; -use tokio::time::Duration; -use tokio_util::io::StreamReader; -use tonic::transport::Channel; use uuid::Uuid; use crate::auth::parse_jwt_keys; use crate::connection::config::DatabaseConfig; -use crate::connection::connection_manager::InnerWalManager; -use crate::connection::libsql::{open_conn, MakeLibSqlConn}; -use crate::connection::{Connection as _, MakeConnection}; -use crate::database::{ - Database, DatabaseKind, PrimaryConnection, PrimaryConnectionMaker, -}; -use crate::error::LoadDumpError; -use crate::replication::script_backup_manager::ScriptBackupManager; -use crate::replication::{FrameNo, ReplicationLogger}; -use crate::schema::SchedulerHandle; +use crate::connection::Connection as _; +use crate::database::Database; use crate::stats::Stats; -use crate::{ - StatsSender, BLOCKING_RT, DB_CREATE_TIMEOUT, DEFAULT_AUTO_CHECKPOINT, -}; -pub use fork::ForkError; - -use self::fork::{ForkTask, PointInTimeRestore}; use self::meta_store::MetaStoreHandle; pub use self::name::NamespaceName; -use self::replication_wal::{make_replication_wal_wrapper, ReplicationWalWrapper}; pub use self::store::NamespaceStore; pub mod broadcasters; -mod fork; pub mod meta_store; mod name; pub mod replication_wal; @@ -101,51 +73,6 @@ impl Namespace { &self.name } - /// completely remove resources associated with the namespace - pub(crate) async fn cleanup( - ns_config: &NamespaceConfig, - name: &NamespaceName, - db_config: &DatabaseConfig, - prune_all: bool, - bottomless_db_id_init: NamespaceBottomlessDbIdInit, - ) -> crate::Result<()> { - let ns_path = ns_config.base_path.join("dbs").join(name.as_str()); - match ns_config.db_kind { - DatabaseKind::Primary => { - if let Some(ref options) = ns_config.bottomless_replication { - let bottomless_db_id = match bottomless_db_id_init { - NamespaceBottomlessDbIdInit::Provided(db_id) => db_id, - NamespaceBottomlessDbIdInit::FetchFromConfig => { - NamespaceBottomlessDbId::from_config(&db_config) - } - }; - let options = make_bottomless_options(options, bottomless_db_id, name.clone()); - let replicator = bottomless::replicator::Replicator::with_options( - ns_path.join("data").to_str().unwrap(), - options, - ) - .await?; - if prune_all { - let delete_all = replicator.delete_all(None).await?; - // perform hard deletion in the background - tokio::spawn(delete_all.commit()); - } else { - // for soft delete make sure that local db is fully backed up - replicator.savepoint().confirmed().await?; - } - } - } - DatabaseKind::Replica => (), - } - - if ns_path.try_exists()? { - tracing::debug!("removing database directory: {}", ns_path.display()); - tokio::fs::remove_dir_all(ns_path).await?; - } - - Ok(()) - } - async fn destroy(mut self) -> anyhow::Result<()> { self.tasks.shutdown().await; self.db.destroy(); @@ -195,293 +122,11 @@ impl Namespace { pub fn config_changed(&self) -> impl Future { self.db_config_store.changed() } - - #[tracing::instrument(skip_all)] - async fn make_primary_connection_maker( - ns_config: &NamespaceConfig, - meta_store_handle: &MetaStoreHandle, - db_path: &Path, - name: &NamespaceName, - restore_option: RestoreOption, - block_writes: Arc, - join_set: &mut JoinSet>, - resolve_attach_path: ResolveNamespacePathFn, - broadcaster: BroadcasterHandle, - ) -> crate::Result<(PrimaryConnectionMaker, ReplicationWalWrapper, Arc)> { - let db_config = meta_store_handle.get(); - let bottomless_db_id = NamespaceBottomlessDbId::from_config(&db_config); - // FIXME: figure how to to it per-db - let mut is_dirty = { - let sentinel_path = db_path.join(".sentinel"); - if sentinel_path.try_exists()? { - true - } else { - tokio::fs::File::create(&sentinel_path).await?; - false - } - }; - - // FIXME: due to a bug in logger::checkpoint_db we call regular checkpointing code - // instead of our virtual WAL one. It's a bit tangled to fix right now, because - // we need WAL context for checkpointing, and WAL context needs the ReplicationLogger... - // So instead we checkpoint early, *before* bottomless gets initialized. That way - // we're sure bottomless won't try to back up any existing WAL frames and will instead - // treat the existing db file as the source of truth. - - let bottomless_replicator = match ns_config.bottomless_replication { - Some(ref options) => { - tracing::debug!("Checkpointing before initializing bottomless"); - crate::replication::primary::logger::checkpoint_db(&db_path.join("data"))?; - tracing::debug!("Checkpointed before initializing bottomless"); - let options = make_bottomless_options(options, bottomless_db_id, name.clone()); - let (replicator, did_recover) = - init_bottomless_replicator(db_path.join("data"), options, &restore_option) - .await?; - tracing::debug!("Completed init of bottomless replicator"); - is_dirty |= did_recover; - Some(replicator) - } - None => None, - }; - - tracing::debug!("Checking fresh db"); - let is_fresh_db = check_fresh_db(&db_path)?; - // switch frame-count checkpoint to time-based one - let auto_checkpoint = if ns_config.checkpoint_interval.is_some() { - 0 - } else { - DEFAULT_AUTO_CHECKPOINT - }; - - let logger = Arc::new(ReplicationLogger::open( - &db_path, - ns_config.max_log_size, - ns_config.max_log_duration, - is_dirty, - auto_checkpoint, - ns_config.scripted_backup.clone(), - name.clone(), - ns_config.encryption_config.clone(), - )?); - - tracing::debug!("sending stats"); - - let stats = make_stats( - &db_path, - join_set, - meta_store_handle.clone(), - ns_config.stats_sender.clone(), - name.clone(), - logger.new_frame_notifier.subscribe(), - ns_config.encryption_config.clone(), - ) - .await?; - - tracing::debug!("Making replication wal wrapper"); - let wal_wrapper = make_replication_wal_wrapper(bottomless_replicator, logger.clone()); - - tracing::debug!("Opening libsql connection"); - - let connection_maker = MakeLibSqlConn::new( - db_path.to_path_buf(), - wal_wrapper.clone(), - stats.clone(), - broadcaster, - meta_store_handle.clone(), - ns_config.extensions.clone(), - ns_config.max_response_size, - ns_config.max_total_response_size, - auto_checkpoint, - logger.new_frame_notifier.subscribe(), - ns_config.encryption_config.clone(), - block_writes, - resolve_attach_path, - ns_config.make_wal_manager.clone(), - ) - .await? - .throttled( - ns_config.max_concurrent_connections.clone(), - Some(DB_CREATE_TIMEOUT), - ns_config.max_total_response_size, - ns_config.max_concurrent_requests, - ); - - tracing::debug!("Completed opening libsql connection"); - - // this must happen after we create the connection maker. The connection maker old on a - // connection to ensure that no other connection is closing while we try to open the dump. - // that would cause a SQLITE_LOCKED error. - match restore_option { - RestoreOption::Dump(_) if !is_fresh_db => { - Err(LoadDumpError::LoadDumpExistingDb)?; - } - RestoreOption::Dump(dump) => { - let conn = connection_maker.create().await?; - tracing::debug!("Loading dump"); - load_dump(dump, conn).await?; - tracing::debug!("Done loading dump"); - } - _ => { /* other cases were already handled when creating bottomless */ } - } - - join_set.spawn(run_periodic_compactions(logger.clone())); - - tracing::debug!("Done making primary connection"); - - Ok((connection_maker, wal_wrapper, stats)) - } - - async fn fork( - ns_config: &NamespaceConfig, - from_ns: &Namespace, - from_config: MetaStoreHandle, - to_ns: NamespaceName, - to_config: MetaStoreHandle, - timestamp: Option, - store: NamespaceStore, - ) -> crate::Result { - let from_config = from_config.get(); - match ns_config.db_kind { - DatabaseKind::Primary => { - let bottomless_db_id = NamespaceBottomlessDbId::from_config(&from_config); - let restore_to = if let Some(timestamp) = timestamp { - if let Some(ref options) = ns_config.bottomless_replication { - Some(PointInTimeRestore { - timestamp, - replicator_options: make_bottomless_options( - options, - bottomless_db_id.clone(), - from_ns.name().clone(), - ), - }) - } else { - return Err(crate::Error::Fork(ForkError::BackupServiceNotConfigured)); - } - } else { - None - }; - - let logger = match &from_ns.db { - Database::Primary(db) => db.wal_wrapper.wrapper().logger(), - Database::Schema(db) => db.wal_wrapper.wrapper().logger(), - _ => { - return Err(crate::Error::Fork(ForkError::Internal(Error::msg( - "Invalid source database type for fork", - )))); - } - }; - - let fork_task = ForkTask { - base_path: ns_config.base_path.clone(), - to_namespace: to_ns.clone(), - logger, - restore_to, - to_config, - store, - }; - - let ns = fork_task.fork().await?; - Ok(ns) - } - DatabaseKind::Replica => Err(ForkError::ForkReplica.into()), - } - } -} - -pub struct NamespaceConfig { - /// Default database kind the store should be Creating - pub(crate) db_kind: DatabaseKind, - // Common config - pub(crate) base_path: Arc, - pub(crate) max_log_size: u64, - pub(crate) max_log_duration: Option, - pub(crate) extensions: Arc<[PathBuf]>, - pub(crate) stats_sender: StatsSender, - pub(crate) max_response_size: u64, - pub(crate) max_total_response_size: u64, - pub(crate) checkpoint_interval: Option, - pub(crate) max_concurrent_connections: Arc, - pub(crate) max_concurrent_requests: u64, - pub(crate) encryption_config: Option, - - // Replica specific config - /// grpc channel for replica - pub channel: Option, - /// grpc uri - pub uri: Option, - - // primary only config - pub(crate) bottomless_replication: Option, - pub(crate) scripted_backup: Option, - pub(crate) migration_scheduler: SchedulerHandle, - pub(crate) make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, } pub type DumpStream = Box> + Send + Sync + 'static + Unpin>; -fn make_bottomless_options( - options: &Options, - namespace_db_id: NamespaceBottomlessDbId, - name: NamespaceName, -) -> Options { - let mut options = options.clone(); - let mut db_id = match namespace_db_id { - NamespaceBottomlessDbId::Namespace(id) => id, - // FIXME(marin): I don't like that, if bottomless is enabled, proper config must be passed. - NamespaceBottomlessDbId::NotProvided => options.db_id.unwrap_or_default(), - }; - - db_id = format!("ns-{db_id}:{name}"); - options.db_id = Some(db_id); - options -} - -async fn make_stats( - db_path: &Path, - join_set: &mut JoinSet>, - meta_store_handle: MetaStoreHandle, - stats_sender: StatsSender, - name: NamespaceName, - mut current_frame_no: watch::Receiver>, - encryption_config: Option, -) -> anyhow::Result> { - tracing::debug!("creating stats type"); - let stats = Stats::new(name.clone(), db_path, join_set).await?; - - // the storage monitor is optional, so we ignore the error here. - tracing::debug!("stats created, sending stats"); - let _ = stats_sender - .send((name.clone(), meta_store_handle, Arc::downgrade(&stats))) - .await; - - join_set.spawn({ - let stats = stats.clone(); - // initialize the current_frame_no value - current_frame_no - .borrow_and_update() - .map(|fno| stats.set_current_frame_no(fno)); - async move { - while current_frame_no.changed().await.is_ok() { - current_frame_no - .borrow_and_update() - .map(|fno| stats.set_current_frame_no(fno)); - } - Ok(()) - } - }); - - join_set.spawn(run_storage_monitor( - db_path.into(), - Arc::downgrade(&stats), - encryption_config, - )); - - tracing::debug!("done sending stats, and creating bg tasks"); - - Ok(stats) -} - #[derive(Default)] pub enum RestoreOption { /// Restore database state from the most recent version found in a backup. @@ -495,189 +140,3 @@ pub enum RestoreOption { /// Granularity depends of how frequently WAL log pages are being snapshotted. PointInTime(NaiveDateTime), } - -const WASM_TABLE_CREATE: &str = - "CREATE TABLE libsql_wasm_func_table (name text PRIMARY KEY, body text) WITHOUT ROWID;"; - -async fn load_dump(dump: S, conn: PrimaryConnection) -> crate::Result<(), LoadDumpError> -where - S: Stream> + Unpin, -{ - let mut reader = tokio::io::BufReader::new(StreamReader::new(dump)); - let mut curr = String::new(); - let mut line = String::new(); - let mut skipped_wasm_table = false; - let mut n_stmt = 0; - let mut line_id = 0; - - while let Ok(n) = reader.read_line(&mut curr).await { - line_id += 1; - if n == 0 { - break; - } - let trimmed = curr.trim(); - if trimmed.is_empty() || trimmed.starts_with("--") { - curr.clear(); - continue; - } - // FIXME: it's well known bug that comment ending with semicolon will be handled incorrectly by currend dump processing code - let statement_end = trimmed.ends_with(';'); - - // we want to concat original(non-trimmed) lines as trimming will join all them in one - // single-line statement which is incorrect if comments in the end are present - line.push_str(&curr); - curr.clear(); - - // This is a hack to ignore the libsql_wasm_func_table table because it is already created - // by the system. - if !skipped_wasm_table && line.trim() == WASM_TABLE_CREATE { - skipped_wasm_table = true; - line.clear(); - continue; - } - - if statement_end { - n_stmt += 1; - // dump must be performd within a txn - if n_stmt > 2 && conn.is_autocommit().await.unwrap() { - return Err(LoadDumpError::NoTxn); - } - - line = tokio::task::spawn_blocking({ - let conn = conn.clone(); - move || -> crate::Result { - conn.with_raw(|conn| conn.execute(&line, ())).map_err(|e| { - LoadDumpError::Internal(format!("line: {}, error: {}", line_id, e)) - })?; - Ok(line) - } - }) - .await??; - line.clear(); - } else { - line.push(' '); - } - } - tracing::debug!("loaded {} lines from dump", line_id); - - if !conn.is_autocommit().await.unwrap() { - tokio::task::spawn_blocking({ - let conn = conn.clone(); - move || -> crate::Result<(), LoadDumpError> { - conn.with_raw(|conn| conn.execute("rollback", ()))?; - Ok(()) - } - }) - .await??; - return Err(LoadDumpError::NoCommit); - } - - Ok(()) -} - -pub async fn init_bottomless_replicator( - path: impl AsRef, - options: bottomless::replicator::Options, - restore_option: &RestoreOption, -) -> anyhow::Result<(bottomless::replicator::Replicator, bool)> { - tracing::debug!("Initializing bottomless replication"); - let path = path - .as_ref() - .to_str() - .ok_or_else(|| anyhow::anyhow!("Invalid db path"))? - .to_owned(); - let mut replicator = bottomless::replicator::Replicator::with_options(path, options).await?; - - let (generation, timestamp) = match restore_option { - RestoreOption::Latest | RestoreOption::Dump(_) => (None, None), - RestoreOption::Generation(generation) => (Some(*generation), None), - RestoreOption::PointInTime(timestamp) => (None, Some(*timestamp)), - }; - - let (action, did_recover) = replicator.restore(generation, timestamp).await?; - match action { - bottomless::replicator::RestoreAction::SnapshotMainDbFile => { - replicator.new_generation().await; - if let Some(_handle) = replicator.snapshot_main_db_file(true).await? { - tracing::trace!("got snapshot handle after restore with generation upgrade"); - } - // Restoration process only leaves the local WAL file if it was - // detected to be newer than its remote counterpart. - replicator.maybe_replicate_wal().await? - } - bottomless::replicator::RestoreAction::ReuseGeneration(gen) => { - replicator.set_generation(gen); - } - } - - Ok((replicator, did_recover)) -} - -async fn run_periodic_compactions(logger: Arc) -> anyhow::Result<()> { - // calling `ReplicationLogger::maybe_compact()` is cheap if the compaction does not actually - // take place, so we can afford to poll it very often for simplicity - let mut interval = tokio::time::interval(tokio::time::Duration::from_millis(1000)); - interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); - - loop { - interval.tick().await; - let handle = BLOCKING_RT.spawn_blocking(enclose! {(logger) move || { - logger.maybe_compact() - }}); - handle - .await - .expect("Compaction task crashed") - .context("Compaction failed")?; - } -} - -fn check_fresh_db(path: &Path) -> crate::Result { - let is_fresh = !path.join("wallog").try_exists()?; - Ok(is_fresh) -} - -// Periodically check the storage used by the database and save it in the Stats structure. -// TODO: Once we have a separate fiber that does WAL checkpoints, running this routine -// right after checkpointing is exactly where it should be done. -async fn run_storage_monitor( - db_path: PathBuf, - stats: Weak, - encryption_config: Option, -) -> anyhow::Result<()> { - // on initialization, the database file doesn't exist yet, so we wait a bit for it to be - // created - tokio::time::sleep(Duration::from_secs(1)).await; - - let duration = tokio::time::Duration::from_secs(60); - let db_path: Arc = db_path.into(); - loop { - let db_path = db_path.clone(); - let Some(stats) = stats.upgrade() else { - return Ok(()); - }; - - let encryption_config = encryption_config.clone(); - let _ = tokio::task::spawn_blocking(move || { - // because closing the last connection interferes with opening a new one, we lazily - // initialize a connection here, and keep it alive for the entirety of the program. If we - // fail to open it, we wait for `duration` and try again later. - match open_conn(&db_path, Sqlite3WalManager::new(), Some(rusqlite::OpenFlags::SQLITE_OPEN_READ_ONLY), encryption_config) { - Ok(mut conn) => { - if let Ok(tx) = conn.transaction() { - let page_count = tx.query_row("pragma page_count;", [], |row| { row.get::(0) }); - let freelist_count = tx.query_row("pragma freelist_count;", [], |row| { row.get::(0) }); - if let (Ok(page_count), Ok(freelist_count)) = (page_count, freelist_count) { - let storage_bytes_used = (page_count - freelist_count) * 4096; - stats.set_storage_bytes_used(storage_bytes_used); - } - } - }, - Err(e) => { - tracing::warn!("failed to open connection for storager monitor: {e}, trying again in {duration:?}"); - }, - } - }).await; - - tokio::time::sleep(duration).await; - } -} diff --git a/libsql-server/src/namespace/store.rs b/libsql-server/src/namespace/store.rs index fbce8cd78b..a78e4f59b0 100644 --- a/libsql-server/src/namespace/store.rs +++ b/libsql-server/src/namespace/store.rs @@ -20,10 +20,10 @@ use crate::namespace::{NamespaceBottomlessDbId, NamespaceBottomlessDbIdInit, Nam use crate::stats::Stats; use super::broadcasters::{BroadcasterHandle, BroadcasterRegistry}; -use super::configurator::NamespaceConfigurators; +use super::configurator::{DynConfigurator, NamespaceConfigurators}; use super::meta_store::{MetaStore, MetaStoreHandle}; use super::schema_lock::SchemaLocksRegistry; -use super::{Namespace, NamespaceConfig, ResetCb, ResetOp, ResolveNamespacePathFn, RestoreOption}; +use super::{Namespace, ResetCb, ResetOp, ResolveNamespacePathFn, RestoreOption}; type NamespaceEntry = Arc>>; @@ -46,10 +46,10 @@ pub struct NamespaceStoreInner { allow_lazy_creation: bool, has_shutdown: AtomicBool, snapshot_at_shutdown: bool, - pub config: NamespaceConfig, schema_locks: SchemaLocksRegistry, broadcasters: BroadcasterRegistry, configurators: NamespaceConfigurators, + db_kind: DatabaseKind, } impl NamespaceStore { @@ -57,9 +57,9 @@ impl NamespaceStore { allow_lazy_creation: bool, snapshot_at_shutdown: bool, max_active_namespaces: usize, - config: NamespaceConfig, metadata: MetaStore, configurators: NamespaceConfigurators, + db_kind: DatabaseKind, ) -> crate::Result { tracing::trace!("Max active namespaces: {max_active_namespaces}"); let store = Cache::::builder() @@ -91,10 +91,10 @@ impl NamespaceStore { allow_lazy_creation, has_shutdown: AtomicBool::new(false), snapshot_at_shutdown, - config, schema_locks: Default::default(), broadcasters: Default::default(), configurators, + db_kind, }), }) } @@ -132,14 +132,8 @@ impl NamespaceStore { } } - Namespace::cleanup( - &self.inner.config, - &namespace, - &db_config, - prune_all, - bottomless_db_id_init, - ) - .await?; + self.cleanup(&namespace, &db_config, prune_all, bottomless_db_id_init) + .await?; tracing::info!("destroyed namespace: {namespace}"); @@ -181,15 +175,16 @@ impl NamespaceStore { let db_config = self.inner.metadata.handle(namespace.clone()); // destroy on-disk database - Namespace::cleanup( - &self.inner.config, + self.cleanup( &namespace, &db_config.get(), false, NamespaceBottomlessDbIdInit::FetchFromConfig, ) .await?; - let ns = self.make_namespace(&namespace, db_config, restore_option).await?; + let ns = self + .make_namespace(&namespace, db_config, restore_option) + .await?; lock.replace(ns); @@ -289,16 +284,17 @@ impl NamespaceStore { handle .store_and_maybe_flush(Some(to_config.into()), false) .await?; - let to_ns = Namespace::fork( - &self.inner.config, - from_ns, - from_config, - to.clone(), - handle.clone(), - timestamp, - self.clone(), - ) - .await?; + let to_ns = self + .get_configurator(&from_config.get()) + .fork( + from_ns, + from_config, + to.clone(), + handle.clone(), + timestamp, + self.clone(), + ) + .await?; to_lock.replace(to_ns); handle.flush().await?; @@ -377,23 +373,18 @@ impl NamespaceStore { config: MetaStoreHandle, restore_option: RestoreOption, ) -> crate::Result { - let configurator = match self.inner.config.db_kind { - DatabaseKind::Primary if config.get().is_shared_schema => { - self.inner.configurators.configure_schema()? - } - DatabaseKind::Primary => self.inner.configurators.configure_primary()?, - DatabaseKind::Replica => self.inner.configurators.configure_replica()?, - }; - let ns = configurator.setup( - &self.inner.config, - config, - restore_option, - namespace, - self.make_reset_cb(), - self.resolve_attach_fn(), - self.clone(), - self.broadcaster(namespace.clone()), - ).await?; + let ns = self + .get_configurator(&config.get()) + .setup( + config, + restore_option, + namespace, + self.make_reset_cb(), + self.resolve_attach_fn(), + self.clone(), + self.broadcaster(namespace.clone()), + ) + .await?; Ok(ns) } @@ -405,7 +396,9 @@ impl NamespaceStore { restore_option: RestoreOption, ) -> crate::Result { let init = async { - let ns = self.make_namespace(namespace, db_config, restore_option).await?; + let ns = self + .make_namespace(namespace, db_config, restore_option) + .await?; Ok(Some(ns)) }; @@ -521,4 +514,26 @@ impl NamespaceStore { pub(crate) fn schema_locks(&self) -> &SchemaLocksRegistry { &self.inner.schema_locks } + + fn get_configurator(&self, db_config: &DatabaseConfig) -> &DynConfigurator { + match self.inner.db_kind { + DatabaseKind::Primary if db_config.is_shared_schema => { + self.inner.configurators.configure_schema().unwrap() + } + DatabaseKind::Primary => self.inner.configurators.configure_primary().unwrap(), + DatabaseKind::Replica => self.inner.configurators.configure_replica().unwrap(), + } + } + + async fn cleanup( + &self, + namespace: &NamespaceName, + db_config: &DatabaseConfig, + prune_all: bool, + bottomless_db_id_init: NamespaceBottomlessDbIdInit, + ) -> crate::Result<()> { + self.get_configurator(db_config) + .cleanup(namespace, db_config, prune_all, bottomless_db_id_init) + .await + } } diff --git a/libsql-server/src/schema/scheduler.rs b/libsql-server/src/schema/scheduler.rs index 17ce655064..a8195cbbd0 100644 --- a/libsql-server/src/schema/scheduler.rs +++ b/libsql-server/src/schema/scheduler.rs @@ -809,10 +809,11 @@ mod test { use crate::connection::config::DatabaseConfig; use crate::database::DatabaseKind; use crate::namespace::configurator::{ - NamespaceConfigurators, PrimaryConfigurator, SchemaConfigurator, + BaseNamespaceConfig, NamespaceConfigurators, PrimaryConfigurator, PrimaryExtraConfig, + SchemaConfigurator, }; use crate::namespace::meta_store::{metastore_connection_maker, MetaStore}; - use crate::namespace::{NamespaceConfig, RestoreOption}; + use crate::namespace::RestoreOption; use crate::schema::SchedulerHandle; use super::super::migration::has_pending_migration_task; @@ -833,9 +834,9 @@ mod test { false, false, 10, - config, meta_store, - NamespaceConfigurators::default(), + config, + DatabaseKind::Primary ) .await .unwrap(); @@ -912,27 +913,41 @@ mod test { assert!(!block_write.load(std::sync::atomic::Ordering::Relaxed)); } - fn make_config(migration_scheduler: SchedulerHandle, path: &Path) -> NamespaceConfig { - NamespaceConfig { - db_kind: DatabaseKind::Primary, + fn make_config(migration_scheduler: SchedulerHandle, path: &Path) -> NamespaceConfigurators { + let mut configurators = NamespaceConfigurators::empty(); + let base_config = BaseNamespaceConfig { base_path: path.to_path_buf().into(), - max_log_size: 1000000000, - max_log_duration: None, extensions: Arc::new([]), stats_sender: tokio::sync::mpsc::channel(1).0, max_response_size: 100000000000000, max_total_response_size: 100000000000, - checkpoint_interval: None, max_concurrent_connections: Arc::new(Semaphore::new(10)), max_concurrent_requests: 10000, - encryption_config: None, - channel: None, - uri: None, + }; + + let primary_config = PrimaryExtraConfig { + max_log_size: 1000000000, + max_log_duration: None, bottomless_replication: None, scripted_backup: None, + checkpoint_interval: None, + }; + + let make_wal_manager = Arc::new(|| EitherWAL::A(Sqlite3WalManager::default())); + + configurators.with_schema(SchemaConfigurator::new( + base_config.clone(), + primary_config.clone(), + make_wal_manager.clone(), migration_scheduler, - make_wal_manager: Arc::new(|| EitherWAL::A(Sqlite3WalManager::default())), - } + )); + configurators.with_primary(PrimaryConfigurator::new( + base_config, + primary_config, + make_wal_manager.clone(), + )); + + configurators } #[tokio::test] @@ -950,9 +965,9 @@ mod test { false, false, 10, - config, meta_store, - NamespaceConfigurators::default(), + config, + DatabaseKind::Primary ) .await .unwrap(); @@ -1029,9 +1044,16 @@ mod test { .unwrap(); let (sender, _receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new(false, false, 10, config, meta_store, NamespaceConfigurators::default()) - .await - .unwrap(); + let store = NamespaceStore::new( + false, + false, + 10, + meta_store, + config, + DatabaseKind::Primary, + ) + .await + .unwrap(); store .with("ns".into(), |ns| { @@ -1056,10 +1078,7 @@ mod test { .unwrap(); let (sender, mut receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let configurators = NamespaceConfigurators::default() - .with_schema(SchemaConfigurator) - .with_primary(PrimaryConfigurator); - let store = NamespaceStore::new(false, false, 10, config, meta_store, configurators) + let store = NamespaceStore::new(false, false, 10, meta_store, config, DatabaseKind::Primary) .await .unwrap(); let mut scheduler = Scheduler::new(store.clone(), maker().unwrap()) @@ -1132,9 +1151,16 @@ mod test { .unwrap(); let (sender, _receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new(false, false, 10, config, meta_store, NamespaceConfigurators::default()) - .await - .unwrap(); + let store = NamespaceStore::new( + false, + false, + 10, + meta_store, + config, + DatabaseKind::Primary + ) + .await + .unwrap(); let scheduler = Scheduler::new(store.clone(), maker().unwrap()) .await .unwrap(); From 0647711dd81736bcb1fa1f886b3076736becc4a9 Mon Sep 17 00:00:00 2001 From: ad hoc Date: Tue, 6 Aug 2024 08:30:53 +0200 Subject: [PATCH 07/13] legacy configurators --- libsql-server/src/http/admin/stats.rs | 2 + libsql-server/src/lib.rs | 425 +++++++++++++++----------- libsql-server/src/namespace/store.rs | 13 +- libsql-server/tests/cluster/mod.rs | 29 +- 4 files changed, 279 insertions(+), 190 deletions(-) diff --git a/libsql-server/src/http/admin/stats.rs b/libsql-server/src/http/admin/stats.rs index f2948d4d7b..5fce92ba0a 100644 --- a/libsql-server/src/http/admin/stats.rs +++ b/libsql-server/src/http/admin/stats.rs @@ -140,10 +140,12 @@ pub(super) async fn handle_stats( State(app_state): State>>, Path(namespace): Path, ) -> crate::Result> { + dbg!(); let stats = app_state .namespaces .stats(NamespaceName::from_string(namespace)?) .await?; + dbg!(); let resp: StatsResponse = stats.as_ref().into(); Ok(Json(resp)) diff --git a/libsql-server/src/lib.rs b/libsql-server/src/lib.rs index 8bd3ea4fac..4188365e03 100644 --- a/libsql-server/src/lib.rs +++ b/libsql-server/src/lib.rs @@ -4,7 +4,6 @@ use std::alloc::Layout; use std::ffi::c_void; use std::mem::{align_of, size_of}; use std::path::{Path, PathBuf}; -use std::pin::Pin; use std::str::FromStr; use std::sync::{Arc, Weak}; @@ -29,10 +28,10 @@ use auth::Auth; use config::{ AdminApiConfig, DbConfig, HeartbeatConfig, RpcClientConfig, RpcServerConfig, UserApiConfig, }; -use futures::future::ready; use futures::Future; use http::user::UserApi; use hyper::client::HttpConnector; +use hyper::Uri; use hyper_rustls::HttpsConnector; #[cfg(feature = "durable-wal")] use libsql_storage::{DurableWalManager, LockManager}; @@ -41,10 +40,6 @@ use libsql_sys::wal::either::Either as EitherWAL; #[cfg(feature = "durable-wal")] use libsql_sys::wal::either::Either3 as EitherWAL; use libsql_sys::wal::Sqlite3WalManager; -use libsql_wal::checkpointer::LibsqlCheckpointer; -use libsql_wal::registry::WalRegistry; -use libsql_wal::storage::NoStorage; -use libsql_wal::wal::LibsqlWalManager; use namespace::meta_store::MetaStoreHandle; use namespace::NamespaceName; use net::Connector; @@ -55,15 +50,19 @@ use tokio::runtime::Runtime; use tokio::sync::{mpsc, Notify, Semaphore}; use tokio::task::JoinSet; use tokio::time::Duration; +use tonic::transport::Channel; use url::Url; use utils::services::idle_shutdown::IdleShutdownKicker; use self::config::MetaStoreConfig; -use self::connection::connection_manager::InnerWalManager; -use self::namespace::configurator::{BaseNamespaceConfig, NamespaceConfigurators, PrimaryConfigurator, PrimaryExtraConfig, ReplicaConfigurator, SchemaConfigurator}; +use self::namespace::configurator::{ + BaseNamespaceConfig, NamespaceConfigurators, PrimaryConfigurator, PrimaryExtraConfig, + ReplicaConfigurator, SchemaConfigurator, +}; use self::namespace::NamespaceStore; use self::net::AddrIncoming; use self::replication::script_backup_manager::{CommandHandler, ScriptBackupManager}; +use self::schema::SchedulerHandle; pub mod auth; mod broadcaster; @@ -424,33 +423,44 @@ where let extensions = self.db_config.validate_extensions()?; let user_auth_strategy = self.user_api_config.auth_strategy.clone(); - let service_shutdown = Arc::new(Notify::new()); - let scripted_backup = match self.db_config.snapshot_exec { Some(ref command) => { let (scripted_backup, script_backup_task) = ScriptBackupManager::new(&self.path, CommandHandler::new(command.to_string())) .await?; - join_set.spawn(script_backup_task.run()); + self.spawn_until_shutdown(&mut join_set, script_backup_task.run()); Some(scripted_backup) } None => None, }; - let (channel, uri) = match self.rpc_client_config { - Some(ref config) => { - let (channel, uri) = config.configure().await?; - (Some(channel), Some(uri)) - } - None => (None, None), + let db_kind = match self.rpc_client_config { + Some(_) => DatabaseKind::Replica, + _ => DatabaseKind::Primary, }; + let client_config = self.get_client_config().await?; let (scheduler_sender, scheduler_receiver) = mpsc::channel(128); - let (stats_sender, stats_receiver) = mpsc::channel(1024); - // chose the wal backend - let (make_wal_manager, registry_shutdown) = self.configure_wal_manager(&mut join_set)?; + let base_config = BaseNamespaceConfig { + base_path: self.path.clone(), + extensions, + stats_sender, + max_response_size: self.db_config.max_response_size, + max_total_response_size: self.db_config.max_total_response_size, + max_concurrent_connections: Arc::new(Semaphore::new(self.max_concurrent_connections)), + max_concurrent_requests: self.db_config.max_concurrent_requests, + }; + + let configurators = self + .make_configurators( + base_config, + scripted_backup, + scheduler_sender.into(), + client_config.clone(), + ) + .await?; let (metastore_conn_maker, meta_store_wal_manager) = metastore_connection_maker(self.meta_store_config.bottomless.clone(), &self.path) @@ -464,60 +474,6 @@ where ) .await?; - let base_config = BaseNamespaceConfig { - base_path: self.path.clone(), - extensions, - stats_sender, - max_response_size: self.db_config.max_response_size, - max_total_response_size: self.db_config.max_total_response_size, - max_concurrent_connections: Arc::new(Semaphore::new(self.max_concurrent_connections)), - max_concurrent_requests: self.db_config.max_concurrent_requests, - }; - - let mut configurators = NamespaceConfigurators::default(); - - let db_kind = match channel.clone().zip(uri.clone()) { - // replica mode - Some((channel, uri)) => { - let replica_configurator = ReplicaConfigurator::new( - base_config, - channel, - uri, - make_wal_manager, - ); - configurators.with_replica(replica_configurator); - DatabaseKind::Replica - } - // primary mode - None => { - let primary_config = PrimaryExtraConfig { - max_log_size: self.db_config.max_log_size, - max_log_duration: self.db_config.max_log_duration.map(Duration::from_secs_f32), - bottomless_replication: self.db_config.bottomless_replication.clone(), - scripted_backup, - checkpoint_interval: self.db_config.checkpoint_interval, - }; - - let primary_configurator = PrimaryConfigurator::new( - base_config.clone(), - primary_config.clone(), - make_wal_manager.clone(), - ); - - let schema_configurator = SchemaConfigurator::new( - base_config.clone(), - primary_config, - make_wal_manager.clone(), - scheduler_sender.into(), - ); - - configurators.with_schema(schema_configurator); - configurators.with_primary(primary_configurator); - - DatabaseKind::Primary - }, - }; - let namespace_store: NamespaceStore = NamespaceStore::new( db_kind.is_replica(), self.db_config.snapshot_at_shutdown, @@ -528,27 +484,9 @@ where ) .await?; - let meta_conn = metastore_conn_maker()?; - let scheduler = Scheduler::new(namespace_store.clone(), meta_conn).await?; - - join_set.spawn(async move { - scheduler.run(scheduler_receiver).await; - Ok(()) - }); self.spawn_monitoring_tasks(&mut join_set, stats_receiver)?; - // eagerly load the default namespace when namespaces are disabled - if self.disable_namespaces && db_kind.is_primary() { - namespace_store - .create( - NamespaceName::default(), - namespace::RestoreOption::Latest, - Default::default(), - ) - .await?; - } - // if namespaces are enabled, then bottomless must have set DB ID if !self.disable_namespaces { if let Some(bottomless) = &self.db_config.bottomless_replication { @@ -563,7 +501,7 @@ where let proxy_service = ProxyService::new(namespace_store.clone(), None, self.disable_namespaces); // Garbage collect proxy clients every 30 seconds - join_set.spawn({ + self.spawn_until_shutdown(&mut join_set, { let clients = proxy_service.clients(); async move { loop { @@ -572,7 +510,8 @@ where } } }); - join_set.spawn(run_rpc_server( + + self.spawn_until_shutdown(&mut join_set, run_rpc_server( proxy_service, config.acceptor, config.tls_config, @@ -584,9 +523,28 @@ where let shutdown_timeout = self.shutdown_timeout.clone(); let shutdown = self.shutdown.clone(); + let service_shutdown = Arc::new(Notify::new()); // setup user-facing rpc services match db_kind { DatabaseKind::Primary => { + // The migration scheduler is only useful on the primary + let meta_conn = metastore_conn_maker()?; + let scheduler = Scheduler::new(namespace_store.clone(), meta_conn).await?; + self.spawn_until_shutdown(&mut join_set, async move { + scheduler.run(scheduler_receiver).await; + Ok(()) + }); + + if self.disable_namespaces { + namespace_store + .create( + NamespaceName::default(), + namespace::RestoreOption::Latest, + Default::default(), + ) + .await?; + } + let replication_svc = ReplicationLogService::new( namespace_store.clone(), idle_shutdown_kicker.clone(), @@ -602,7 +560,7 @@ where ); // Garbage collect proxy clients every 30 seconds - join_set.spawn({ + self.spawn_until_shutdown(&mut join_set, { let clients = proxy_svc.clients(); async move { loop { @@ -623,16 +581,19 @@ where .configure(&mut join_set); } DatabaseKind::Replica => { + dbg!(); + let (channel, uri) = client_config.clone().unwrap(); let replication_svc = - ReplicationLogProxyService::new(channel.clone().unwrap(), uri.clone().unwrap()); + ReplicationLogProxyService::new(channel.clone(), uri.clone()); let proxy_svc = ReplicaProxyService::new( - channel.clone().unwrap(), - uri.clone().unwrap(), + channel, + uri, namespace_store.clone(), user_auth_strategy.clone(), self.disable_namespaces, ); + dbg!(); self.make_services( namespace_store.clone(), idle_shutdown_kicker, @@ -642,6 +603,7 @@ where service_shutdown.clone(), ) .configure(&mut join_set); + dbg!(); } }; @@ -651,7 +613,6 @@ where join_set.shutdown().await; service_shutdown.notify_waiters(); namespace_store.shutdown().await?; - registry_shutdown.await?; Ok::<_, crate::Error>(()) }; @@ -680,100 +641,200 @@ where Ok(()) } - fn setup_shutdown(&self) -> Option { - let shutdown_notify = self.shutdown.clone(); - self.idle_shutdown_timeout.map(|d| { - IdleShutdownKicker::new(d, self.initial_idle_shutdown_timeout, shutdown_notify) - }) - } - - fn configure_wal_manager( + async fn make_configurators( &self, - join_set: &mut JoinSet>, - ) -> anyhow::Result<( - Arc InnerWalManager + Sync + Send + 'static>, - Pin> + Send + Sync + 'static>>, - )> { - let wal_path = self.path.join("wals"); - let enable_libsql_wal_test = { - let is_primary = self.rpc_server_config.is_some(); - let is_libsql_wal_test = std::env::var("LIBSQL_WAL_TEST").is_ok(); - is_primary && is_libsql_wal_test - }; - let use_libsql_wal = - self.use_custom_wal == Some(CustomWAL::LibsqlWal) || enable_libsql_wal_test; - if !use_libsql_wal { - if wal_path.try_exists()? { - anyhow::bail!("database was previously setup to use libsql-wal"); - } - } - - if self.use_custom_wal.is_some() { - if self.db_config.bottomless_replication.is_some() { - anyhow::bail!("bottomless not supported with custom WAL"); - } - if self.rpc_client_config.is_some() { - anyhow::bail!("custom WAL not supported in replica mode"); + base_config: BaseNamespaceConfig, + scripted_backup: Option, + migration_scheduler_handle: SchedulerHandle, + client_config: Option<(Channel, Uri)>, + ) -> anyhow::Result { + match self.use_custom_wal { + Some(CustomWAL::LibsqlWal) => self.libsql_wal_configurators(), + #[cfg(feature = "durable-wal")] + Some(CustomWAL::DurableWal) => self.durable_wal_configurators(), + None => { + self.legacy_configurators( + base_config, + scripted_backup, + migration_scheduler_handle, + client_config, + ) + .await } } + } - let namespace_resolver = |path: &Path| { - NamespaceName::from_string( - path.parent() - .unwrap() - .file_name() - .unwrap() - .to_str() - .unwrap() - .to_string(), - ) - .unwrap() - .into() - }; - - match self.use_custom_wal { - Some(CustomWAL::LibsqlWal) => { - let (sender, receiver) = tokio::sync::mpsc::channel(64); - let registry = Arc::new(WalRegistry::new(wal_path, NoStorage, sender)?); - let checkpointer = LibsqlCheckpointer::new(registry.clone(), receiver, 8); - join_set.spawn(async move { - checkpointer.run().await; - Ok(()) - }); + fn libsql_wal_configurators(&self) -> anyhow::Result { + todo!() + } - let wal = LibsqlWalManager::new(registry.clone(), Arc::new(namespace_resolver)); - let shutdown_notify = self.shutdown.clone(); - let shutdown_fut = Box::pin(async move { - shutdown_notify.notified().await; - registry.shutdown().await?; - Ok(()) - }); + #[cfg(feature = "durable-wal")] + fn durable_wal_configurators(&self) -> anyhow::Result { + todo!(); + } - tracing::info!("using libsql wal"); - Ok((Arc::new(move || EitherWAL::B(wal.clone())), shutdown_fut)) + fn spawn_until_shutdown(&self, join_set: &mut JoinSet>, fut: F) + where + F: Future> + Send + 'static, + { + let shutdown = self.shutdown.clone(); + join_set.spawn(async move { + tokio::select! { + _ = shutdown.notified() => Ok(()), + ret = fut => ret } - #[cfg(feature = "durable-wal")] - Some(CustomWAL::DurableWal) => { - tracing::info!("using durable wal"); - let lock_manager = Arc::new(std::sync::Mutex::new(LockManager::new())); - let wal = DurableWalManager::new( - lock_manager, - namespace_resolver, - self.storage_server_address.clone(), - ); - Ok(( - Arc::new(move || EitherWAL::C(wal.clone())), - Box::pin(ready(Ok(()))), - )) + }); + } + + async fn legacy_configurators( + &self, + base_config: BaseNamespaceConfig, + scripted_backup: Option, + migration_scheduler_handle: SchedulerHandle, + client_config: Option<(Channel, Uri)>, + ) -> anyhow::Result { + let make_wal_manager = Arc::new(|| EitherWAL::A(Sqlite3WalManager::default())); + let mut configurators = NamespaceConfigurators::empty(); + + match client_config { + // replica mode + Some((channel, uri)) => { + let replica_configurator = + ReplicaConfigurator::new(base_config, channel, uri, make_wal_manager); + configurators.with_replica(replica_configurator); } + // primary mode None => { - tracing::info!("using sqlite3 wal"); - Ok(( - Arc::new(|| EitherWAL::A(Sqlite3WalManager::default())), - Box::pin(ready(Ok(()))), - )) + let primary_config = PrimaryExtraConfig { + max_log_size: self.db_config.max_log_size, + max_log_duration: self.db_config.max_log_duration.map(Duration::from_secs_f32), + bottomless_replication: self.db_config.bottomless_replication.clone(), + scripted_backup, + checkpoint_interval: self.db_config.checkpoint_interval, + }; + + let primary_configurator = PrimaryConfigurator::new( + base_config.clone(), + primary_config.clone(), + make_wal_manager.clone(), + ); + + let schema_configurator = SchemaConfigurator::new( + base_config.clone(), + primary_config, + make_wal_manager.clone(), + migration_scheduler_handle, + ); + + configurators.with_schema(schema_configurator); + configurators.with_primary(primary_configurator); } } + + Ok(configurators) + } + + fn setup_shutdown(&self) -> Option { + let shutdown_notify = self.shutdown.clone(); + self.idle_shutdown_timeout.map(|d| { + IdleShutdownKicker::new(d, self.initial_idle_shutdown_timeout, shutdown_notify) + }) + } + + // fn configure_wal_manager( + // &self, + // join_set: &mut JoinSet>, + // ) -> anyhow::Result<( + // Arc InnerWalManager + Sync + Send + 'static>, + // Pin> + Send + Sync + 'static>>, + // )> { + // let wal_path = self.path.join("wals"); + // let enable_libsql_wal_test = { + // let is_primary = self.rpc_server_config.is_some(); + // let is_libsql_wal_test = std::env::var("LIBSQL_WAL_TEST").is_ok(); + // is_primary && is_libsql_wal_test + // }; + // let use_libsql_wal = + // self.use_custom_wal == Some(CustomWAL::LibsqlWal) || enable_libsql_wal_test; + // if !use_libsql_wal { + // if wal_path.try_exists()? { + // anyhow::bail!("database was previously setup to use libsql-wal"); + // } + // } + // + // if self.use_custom_wal.is_some() { + // if self.db_config.bottomless_replication.is_some() { + // anyhow::bail!("bottomless not supported with custom WAL"); + // } + // if self.rpc_client_config.is_some() { + // anyhow::bail!("custom WAL not supported in replica mode"); + // } + // } + // + // let namespace_resolver = |path: &Path| { + // NamespaceName::from_string( + // path.parent() + // .unwrap() + // .file_name() + // .unwrap() + // .to_str() + // .unwrap() + // .to_string(), + // ) + // .unwrap() + // .into() + // }; + // + // match self.use_custom_wal { + // Some(CustomWAL::LibsqlWal) => { + // let (sender, receiver) = tokio::sync::mpsc::channel(64); + // let registry = Arc::new(WalRegistry::new(wal_path, NoStorage, sender)?); + // let checkpointer = LibsqlCheckpointer::new(registry.clone(), receiver, 8); + // join_set.spawn(async move { + // checkpointer.run().await; + // Ok(()) + // }); + // + // let wal = LibsqlWalManager::new(registry.clone(), Arc::new(namespace_resolver)); + // let shutdown_notify = self.shutdown.clone(); + // let shutdown_fut = Box::pin(async move { + // shutdown_notify.notified().await; + // registry.shutdown().await?; + // Ok(()) + // }); + // + // tracing::info!("using libsql wal"); + // Ok((Arc::new(move || EitherWAL::B(wal.clone())), shutdown_fut)) + // } + // #[cfg(feature = "durable-wal")] + // Some(CustomWAL::DurableWal) => { + // tracing::info!("using durable wal"); + // let lock_manager = Arc::new(std::sync::Mutex::new(LockManager::new())); + // let wal = DurableWalManager::new( + // lock_manager, + // namespace_resolver, + // self.storage_server_address.clone(), + // ); + // Ok(( + // Arc::new(move || EitherWAL::C(wal.clone())), + // Box::pin(ready(Ok(()))), + // )) + // } + // None => { + // tracing::info!("using sqlite3 wal"); + // Ok(( + // Arc::new(|| EitherWAL::A(Sqlite3WalManager::default())), + // Box::pin(ready(Ok(()))), + // )) + // } + // } + // } + + async fn get_client_config(&self) -> anyhow::Result> { + match self.rpc_client_config { + Some(ref config) => Ok(Some(config.configure().await?)), + None => Ok(None), + } } } diff --git a/libsql-server/src/namespace/store.rs b/libsql-server/src/namespace/store.rs index a78e4f59b0..b2b5d33032 100644 --- a/libsql-server/src/namespace/store.rs +++ b/libsql-server/src/namespace/store.rs @@ -327,6 +327,7 @@ impl NamespaceStore { where Fun: FnOnce(&Namespace) -> R, { + dbg!(); if namespace != NamespaceName::default() && !self.inner.metadata.exists(&namespace) && !self.inner.allow_lazy_creation @@ -334,6 +335,7 @@ impl NamespaceStore { return Err(Error::NamespaceDoesntExist(namespace.to_string())); } + dbg!(); let f = { let name = namespace.clone(); move |ns: NamespaceEntry| async move { @@ -346,7 +348,9 @@ impl NamespaceStore { } }; + dbg!(); let handle = self.inner.metadata.handle(namespace.to_owned()); + dbg!(); f(self .load_namespace(&namespace, handle, RestoreOption::Latest) .await?) @@ -373,6 +377,7 @@ impl NamespaceStore { config: MetaStoreHandle, restore_option: RestoreOption, ) -> crate::Result { + dbg!(); let ns = self .get_configurator(&config.get()) .setup( @@ -386,6 +391,7 @@ impl NamespaceStore { ) .await?; + dbg!(); Ok(ns) } @@ -395,13 +401,17 @@ impl NamespaceStore { db_config: MetaStoreHandle, restore_option: RestoreOption, ) -> crate::Result { + dbg!(); let init = async { + dbg!(); let ns = self .make_namespace(namespace, db_config, restore_option) .await?; + dbg!(); Ok(Some(ns)) }; + dbg!(); let before_load = Instant::now(); let ns = self .inner @@ -410,7 +420,8 @@ impl NamespaceStore { namespace.clone(), init.map_ok(|ns| Arc::new(RwLock::new(ns))), ) - .await?; + .await.map_err(|e| dbg!(e))?; + dbg!(); NAMESPACE_LOAD_LATENCY.record(before_load.elapsed()); Ok(ns) diff --git a/libsql-server/tests/cluster/mod.rs b/libsql-server/tests/cluster/mod.rs index 1171d4a5d0..8f214bd05e 100644 --- a/libsql-server/tests/cluster/mod.rs +++ b/libsql-server/tests/cluster/mod.rs @@ -149,23 +149,29 @@ fn sync_many_replica() { let mut sim = Builder::new() .simulation_duration(Duration::from_secs(1000)) .build(); + dbg!(); make_cluster(&mut sim, NUM_REPLICA, true); + dbg!(); sim.client("client", async { let db = Database::open_remote_with_connector("http://primary:8080", "", TurmoilConnector)?; let conn = db.connect()?; + dbg!(); conn.execute("create table test (x)", ()).await?; + dbg!(); conn.execute("insert into test values (42)", ()).await?; + dbg!(); async fn get_frame_no(url: &str) -> Option { let client = Client::new(); + dbg!(); Some( - client - .get(url) - .await - .unwrap() - .json::() - .await + dbg!(client + .get(url) + .await + .unwrap() + .json::() + .await) .unwrap() .get("replication_index")? .as_u64() @@ -173,6 +179,7 @@ fn sync_many_replica() { ) } + dbg!(); let primary_fno = loop { if let Some(fno) = get_frame_no("http://primary:9090/v1/namespaces/default/stats").await { @@ -180,13 +187,15 @@ fn sync_many_replica() { } }; + dbg!(); // wait for all replicas to sync let mut join_set = JoinSet::new(); for i in 0..NUM_REPLICA { join_set.spawn(async move { let uri = format!("http://replica{i}:9090/v1/namespaces/default/stats"); + dbg!(); loop { - if let Some(replica_fno) = get_frame_no(&uri).await { + if let Some(replica_fno) = dbg!(get_frame_no(&uri).await) { if replica_fno == primary_fno { break; } @@ -196,8 +205,10 @@ fn sync_many_replica() { }); } + dbg!(); while join_set.join_next().await.is_some() {} + dbg!(); for i in 0..NUM_REPLICA { let db = Database::open_remote_with_connector( format!("http://replica{i}:8080"), @@ -212,8 +223,10 @@ fn sync_many_replica() { )); } + dbg!(); let client = Client::new(); + dbg!(); let stats = client .get("http://primary:9090/v1/namespaces/default/stats") .await? @@ -221,12 +234,14 @@ fn sync_many_replica() { .await .unwrap(); + dbg!(); let stat = stats .get("embedded_replica_frames_replicated") .unwrap() .as_u64() .unwrap(); + dbg!(); assert_eq!(stat, 0); Ok(()) From b12431c33a172ba0c96f35cfd35e36ce931b5c00 Mon Sep 17 00:00:00 2001 From: ad hoc Date: Tue, 6 Aug 2024 13:30:21 +0200 Subject: [PATCH 08/13] configure durable wal --- libsql-server/src/lib.rs | 98 ++++++++++++++++++++++++++++++++-------- 1 file changed, 78 insertions(+), 20 deletions(-) diff --git a/libsql-server/src/lib.rs b/libsql-server/src/lib.rs index 4188365e03..9ee8e3b908 100644 --- a/libsql-server/src/lib.rs +++ b/libsql-server/src/lib.rs @@ -55,6 +55,7 @@ use url::Url; use utils::services::idle_shutdown::IdleShutdownKicker; use self::config::MetaStoreConfig; +use self::connection::connection_manager::InnerWalManager; use self::namespace::configurator::{ BaseNamespaceConfig, NamespaceConfigurators, PrimaryConfigurator, PrimaryExtraConfig, ReplicaConfigurator, SchemaConfigurator, @@ -336,7 +337,8 @@ where config.heartbeat_url.as_deref().unwrap_or(""), config.heartbeat_period, ); - join_set.spawn({ + + self.spawn_until_shutdown_on(join_set, { let heartbeat_auth = config.heartbeat_auth.clone(); let heartbeat_period = config.heartbeat_period; let heartbeat_url = if let Some(url) = &config.heartbeat_url { @@ -428,7 +430,7 @@ where let (scripted_backup, script_backup_task) = ScriptBackupManager::new(&self.path, CommandHandler::new(command.to_string())) .await?; - self.spawn_until_shutdown(&mut join_set, script_backup_task.run()); + self.spawn_until_shutdown_on(&mut join_set, script_backup_task.run()); Some(scripted_backup) } None => None, @@ -484,7 +486,6 @@ where ) .await?; - self.spawn_monitoring_tasks(&mut join_set, stats_receiver)?; // if namespaces are enabled, then bottomless must have set DB ID @@ -501,7 +502,7 @@ where let proxy_service = ProxyService::new(namespace_store.clone(), None, self.disable_namespaces); // Garbage collect proxy clients every 30 seconds - self.spawn_until_shutdown(&mut join_set, { + self.spawn_until_shutdown_on(&mut join_set, { let clients = proxy_service.clients(); async move { loop { @@ -511,14 +512,17 @@ where } }); - self.spawn_until_shutdown(&mut join_set, run_rpc_server( - proxy_service, - config.acceptor, - config.tls_config, - idle_shutdown_kicker.clone(), - namespace_store.clone(), - self.disable_namespaces, - )); + self.spawn_until_shutdown_on( + &mut join_set, + run_rpc_server( + proxy_service, + config.acceptor, + config.tls_config, + idle_shutdown_kicker.clone(), + namespace_store.clone(), + self.disable_namespaces, + ), + ); } let shutdown_timeout = self.shutdown_timeout.clone(); @@ -530,7 +534,7 @@ where // The migration scheduler is only useful on the primary let meta_conn = metastore_conn_maker()?; let scheduler = Scheduler::new(namespace_store.clone(), meta_conn).await?; - self.spawn_until_shutdown(&mut join_set, async move { + self.spawn_until_shutdown_on(&mut join_set, async move { scheduler.run(scheduler_receiver).await; Ok(()) }); @@ -560,7 +564,7 @@ where ); // Garbage collect proxy clients every 30 seconds - self.spawn_until_shutdown(&mut join_set, { + self.spawn_until_shutdown_on(&mut join_set, { let clients = proxy_svc.clients(); async move { loop { @@ -583,8 +587,7 @@ where DatabaseKind::Replica => { dbg!(); let (channel, uri) = client_config.clone().unwrap(); - let replication_svc = - ReplicationLogProxyService::new(channel.clone(), uri.clone()); + let replication_svc = ReplicationLogProxyService::new(channel.clone(), uri.clone()); let proxy_svc = ReplicaProxyService::new( channel, uri, @@ -651,7 +654,12 @@ where match self.use_custom_wal { Some(CustomWAL::LibsqlWal) => self.libsql_wal_configurators(), #[cfg(feature = "durable-wal")] - Some(CustomWAL::DurableWal) => self.durable_wal_configurators(), + Some(CustomWAL::DurableWal) => self.durable_wal_configurators( + base_config, + scripted_backup, + migration_scheduler_handle, + client_config, + ), None => { self.legacy_configurators( base_config, @@ -669,11 +677,44 @@ where } #[cfg(feature = "durable-wal")] - fn durable_wal_configurators(&self) -> anyhow::Result { - todo!(); + fn durable_wal_configurators( + &self, + base_config: BaseNamespaceConfig, + scripted_backup: Option, + migration_scheduler_handle: SchedulerHandle, + client_config: Option<(Channel, Uri)>, + ) -> anyhow::Result { + tracing::info!("using durable wal"); + let lock_manager = Arc::new(std::sync::Mutex::new(LockManager::new())); + let namespace_resolver = |path: &Path| { + NamespaceName::from_string( + path.parent() + .unwrap() + .file_name() + .unwrap() + .to_str() + .unwrap() + .to_string(), + ) + .unwrap() + .into() + }; + let wal = DurableWalManager::new( + lock_manager, + namespace_resolver, + self.storage_server_address.clone(), + ); + let make_wal_manager = Arc::new(move || EitherWAL::C(wal.clone())); + self.configurators_common( + client_config, + base_config, + make_wal_manager, + scripted_backup, + migration_scheduler_handle, + ) } - fn spawn_until_shutdown(&self, join_set: &mut JoinSet>, fut: F) + fn spawn_until_shutdown_on(&self, join_set: &mut JoinSet>, fut: F) where F: Future> + Send + 'static, { @@ -694,6 +735,23 @@ where client_config: Option<(Channel, Uri)>, ) -> anyhow::Result { let make_wal_manager = Arc::new(|| EitherWAL::A(Sqlite3WalManager::default())); + self.configurators_common( + client_config, + base_config, + make_wal_manager, + scripted_backup, + migration_scheduler_handle, + ) + } + + fn configurators_common( + &self, + client_config: Option<(Channel, Uri)>, + base_config: BaseNamespaceConfig, + make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, + scripted_backup: Option, + migration_scheduler_handle: SchedulerHandle, + ) -> anyhow::Result { let mut configurators = NamespaceConfigurators::empty(); match client_config { From 066f1527572d3e3012457e588e9d28e8e74656ec Mon Sep 17 00:00:00 2001 From: ad hoc Date: Tue, 6 Aug 2024 14:37:10 +0200 Subject: [PATCH 09/13] configure libsql_wal --- libsql-server/src/lib.rs | 213 ++++++++++++++++++++++++++++++--------- 1 file changed, 165 insertions(+), 48 deletions(-) diff --git a/libsql-server/src/lib.rs b/libsql-server/src/lib.rs index 9ee8e3b908..f5788dcebb 100644 --- a/libsql-server/src/lib.rs +++ b/libsql-server/src/lib.rs @@ -28,6 +28,7 @@ use auth::Auth; use config::{ AdminApiConfig, DbConfig, HeartbeatConfig, RpcClientConfig, RpcServerConfig, UserApiConfig, }; +use futures::future::{pending, ready}; use futures::Future; use http::user::UserApi; use hyper::client::HttpConnector; @@ -40,6 +41,10 @@ use libsql_sys::wal::either::Either as EitherWAL; #[cfg(feature = "durable-wal")] use libsql_sys::wal::either::Either3 as EitherWAL; use libsql_sys::wal::Sqlite3WalManager; +use libsql_wal::checkpointer::LibsqlCheckpointer; +use libsql_wal::registry::WalRegistry; +use libsql_wal::storage::NoStorage; +use libsql_wal::wal::LibsqlWalManager; use namespace::meta_store::MetaStoreHandle; use namespace::NamespaceName; use net::Connector; @@ -458,9 +463,10 @@ where let configurators = self .make_configurators( base_config, - scripted_backup, - scheduler_sender.into(), client_config.clone(), + &mut join_set, + scheduler_sender.into(), + scripted_backup, ) .await?; @@ -596,7 +602,6 @@ where self.disable_namespaces, ); - dbg!(); self.make_services( namespace_store.clone(), idle_shutdown_kicker, @@ -647,42 +652,125 @@ where async fn make_configurators( &self, base_config: BaseNamespaceConfig, - scripted_backup: Option, - migration_scheduler_handle: SchedulerHandle, client_config: Option<(Channel, Uri)>, + join_set: &mut JoinSet>, + migration_scheduler_handle: SchedulerHandle, + scripted_backup: Option, ) -> anyhow::Result { + let wal_path = base_config.base_path.join("wals"); + let enable_libsql_wal_test = { + let is_primary = self.rpc_server_config.is_some(); + let is_libsql_wal_test = std::env::var("LIBSQL_WAL_TEST").is_ok(); + is_primary && is_libsql_wal_test + }; + let use_libsql_wal = + self.use_custom_wal == Some(CustomWAL::LibsqlWal) || enable_libsql_wal_test; + if !use_libsql_wal { + if wal_path.try_exists()? { + anyhow::bail!("database was previously setup to use libsql-wal"); + } + } + + if self.use_custom_wal.is_some() { + if self.db_config.bottomless_replication.is_some() { + anyhow::bail!("bottomless not supported with custom WAL"); + } + if self.rpc_client_config.is_some() { + anyhow::bail!("custom WAL not supported in replica mode"); + } + } + match self.use_custom_wal { - Some(CustomWAL::LibsqlWal) => self.libsql_wal_configurators(), + Some(CustomWAL::LibsqlWal) => self.libsql_wal_configurators( + base_config, + client_config, + join_set, + migration_scheduler_handle, + scripted_backup, + wal_path, + ), #[cfg(feature = "durable-wal")] Some(CustomWAL::DurableWal) => self.durable_wal_configurators( base_config, - scripted_backup, - migration_scheduler_handle, client_config, + migration_scheduler_handle, + scripted_backup, ), None => { self.legacy_configurators( base_config, - scripted_backup, - migration_scheduler_handle, client_config, + migration_scheduler_handle, + scripted_backup, ) .await } } } - fn libsql_wal_configurators(&self) -> anyhow::Result { - todo!() + fn libsql_wal_configurators( + &self, + base_config: BaseNamespaceConfig, + client_config: Option<(Channel, Uri)>, + join_set: &mut JoinSet>, + migration_scheduler_handle: SchedulerHandle, + scripted_backup: Option, + wal_path: PathBuf, + ) -> anyhow::Result { + tracing::info!("using libsql wal"); + let (sender, receiver) = tokio::sync::mpsc::channel(64); + let registry = Arc::new(WalRegistry::new(wal_path, NoStorage, sender)?); + let checkpointer = LibsqlCheckpointer::new(registry.clone(), receiver, 8); + self.spawn_until_shutdown_on(join_set, async move { + checkpointer.run().await; + Ok(()) + }); + + let namespace_resolver = |path: &Path| { + NamespaceName::from_string( + path.parent() + .unwrap() + .file_name() + .unwrap() + .to_str() + .unwrap() + .to_string(), + ) + .unwrap() + .into() + }; + let wal = LibsqlWalManager::new(registry.clone(), Arc::new(namespace_resolver)); + + self.spawn_until_shutdown_with_teardown(join_set, pending(), async move { + registry.shutdown().await?; + Ok(()) + }); + + let make_wal_manager = Arc::new(move || EitherWAL::B(wal.clone())); + let mut configurators = NamespaceConfigurators::empty(); + + match client_config { + Some(_) => todo!("configure replica"), + // configure primary + None => self.configure_primary_common( + base_config, + &mut configurators, + make_wal_manager, + migration_scheduler_handle, + scripted_backup, + ), + } + + Ok(configurators) } #[cfg(feature = "durable-wal")] fn durable_wal_configurators( &self, base_config: BaseNamespaceConfig, - scripted_backup: Option, - migration_scheduler_handle: SchedulerHandle, client_config: Option<(Channel, Uri)>, + migration_scheduler_handle: SchedulerHandle, + scripted_backup: Option, ) -> anyhow::Result { tracing::info!("using durable wal"); let lock_manager = Arc::new(std::sync::Mutex::new(LockManager::new())); @@ -706,22 +794,37 @@ where ); let make_wal_manager = Arc::new(move || EitherWAL::C(wal.clone())); self.configurators_common( - client_config, base_config, + client_config, make_wal_manager, - scripted_backup, migration_scheduler_handle, + scripted_backup, ) } fn spawn_until_shutdown_on(&self, join_set: &mut JoinSet>, fut: F) where F: Future> + Send + 'static, + { + self.spawn_until_shutdown_with_teardown(join_set, fut, ready(Ok(()))) + } + + /// run the passed future until shutdown is called, then call the passed teardown future + fn spawn_until_shutdown_with_teardown( + &self, + join_set: &mut JoinSet>, + fut: F, + teardown: T, + ) where + F: Future> + Send + 'static, + T: Future> + Send + 'static, { let shutdown = self.shutdown.clone(); join_set.spawn(async move { tokio::select! { - _ = shutdown.notified() => Ok(()), + _ = shutdown.notified() => { + teardown.await + }, ret = fut => ret } }); @@ -730,30 +833,29 @@ where async fn legacy_configurators( &self, base_config: BaseNamespaceConfig, - scripted_backup: Option, - migration_scheduler_handle: SchedulerHandle, client_config: Option<(Channel, Uri)>, + migration_scheduler_handle: SchedulerHandle, + scripted_backup: Option, ) -> anyhow::Result { let make_wal_manager = Arc::new(|| EitherWAL::A(Sqlite3WalManager::default())); self.configurators_common( - client_config, base_config, + client_config, make_wal_manager, - scripted_backup, migration_scheduler_handle, + scripted_backup, ) } fn configurators_common( &self, - client_config: Option<(Channel, Uri)>, base_config: BaseNamespaceConfig, + client_config: Option<(Channel, Uri)>, make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, - scripted_backup: Option, migration_scheduler_handle: SchedulerHandle, + scripted_backup: Option, ) -> anyhow::Result { let mut configurators = NamespaceConfigurators::empty(); - match client_config { // replica mode Some((channel, uri)) => { @@ -762,34 +864,49 @@ where configurators.with_replica(replica_configurator); } // primary mode - None => { - let primary_config = PrimaryExtraConfig { - max_log_size: self.db_config.max_log_size, - max_log_duration: self.db_config.max_log_duration.map(Duration::from_secs_f32), - bottomless_replication: self.db_config.bottomless_replication.clone(), - scripted_backup, - checkpoint_interval: self.db_config.checkpoint_interval, - }; + None => self.configure_primary_common( + base_config, + &mut configurators, + make_wal_manager, + migration_scheduler_handle, + scripted_backup, + ), + } - let primary_configurator = PrimaryConfigurator::new( - base_config.clone(), - primary_config.clone(), - make_wal_manager.clone(), - ); + Ok(configurators) + } - let schema_configurator = SchemaConfigurator::new( - base_config.clone(), - primary_config, - make_wal_manager.clone(), - migration_scheduler_handle, - ); + fn configure_primary_common( + &self, + base_config: BaseNamespaceConfig, + configurators: &mut NamespaceConfigurators, + make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, + migration_scheduler_handle: SchedulerHandle, + scripted_backup: Option, + ) { + let primary_config = PrimaryExtraConfig { + max_log_size: self.db_config.max_log_size, + max_log_duration: self.db_config.max_log_duration.map(Duration::from_secs_f32), + bottomless_replication: self.db_config.bottomless_replication.clone(), + scripted_backup, + checkpoint_interval: self.db_config.checkpoint_interval, + }; - configurators.with_schema(schema_configurator); - configurators.with_primary(primary_configurator); - } - } + let primary_configurator = PrimaryConfigurator::new( + base_config.clone(), + primary_config.clone(), + make_wal_manager.clone(), + ); - Ok(configurators) + let schema_configurator = SchemaConfigurator::new( + base_config.clone(), + primary_config, + make_wal_manager.clone(), + migration_scheduler_handle, + ); + + configurators.with_schema(schema_configurator); + configurators.with_primary(primary_configurator); } fn setup_shutdown(&self) -> Option { From b5dba7241531d3c5bb58a093189b72b9ea47fb25 Mon Sep 17 00:00:00 2001 From: ad hoc Date: Tue, 6 Aug 2024 16:52:23 +0200 Subject: [PATCH 10/13] partial implmentation of LibsqlWalReplicationConfigurator --- .../configurator/libsql_wal_replica.rs | 138 ++++++++++++++++++ 1 file changed, 138 insertions(+) create mode 100644 libsql-server/src/namespace/configurator/libsql_wal_replica.rs diff --git a/libsql-server/src/namespace/configurator/libsql_wal_replica.rs b/libsql-server/src/namespace/configurator/libsql_wal_replica.rs new file mode 100644 index 0000000000..6b2519cf33 --- /dev/null +++ b/libsql-server/src/namespace/configurator/libsql_wal_replica.rs @@ -0,0 +1,138 @@ +use std::pin::Pin; +use std::future::Future; +use std::sync::Arc; + +use chrono::prelude::NaiveDateTime; +use hyper::Uri; +use libsql_replication::rpc::replication::replication_log_client::ReplicationLogClient; +use libsql_wal::io::StdIO; +use libsql_wal::registry::WalRegistry; +use libsql_wal::storage::NoStorage; +use tokio::task::JoinSet; +use tonic::transport::Channel; + +use crate::connection::config::DatabaseConfig; +use crate::connection::connection_manager::InnerWalManager; +use crate::connection::write_proxy::MakeWriteProxyConn; +use crate::connection::MakeConnection; +use crate::database::{Database, ReplicaDatabase}; +use crate::namespace::broadcasters::BroadcasterHandle; +use crate::namespace::configurator::helpers::make_stats; +use crate::namespace::meta_store::MetaStoreHandle; +use crate::namespace::{ + Namespace, NamespaceBottomlessDbIdInit, NamespaceName, NamespaceStore, ResetCb, + ResolveNamespacePathFn, RestoreOption, +}; +use crate::DEFAULT_AUTO_CHECKPOINT; + +use super::{BaseNamespaceConfig, ConfigureNamespace}; + +pub struct LibsqlWalReplicaConfigurator { + base: BaseNamespaceConfig, + registry: Arc>, + uri: Uri, + channel: Channel, + make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, +} + +impl ConfigureNamespace for LibsqlWalReplicaConfigurator { + fn setup<'a>( + &'a self, + db_config: MetaStoreHandle, + restore_option: RestoreOption, + name: &'a NamespaceName, + reset: ResetCb, + resolve_attach_path: ResolveNamespacePathFn, + store: NamespaceStore, + broadcaster: BroadcasterHandle, + ) -> Pin> + Send + 'a>> { + Box::pin(async move { + tracing::debug!("creating replica namespace"); + let db_path = self.base.base_path.join("dbs").join(name.as_str()); + let channel = self.channel.clone(); + let uri = self.uri.clone(); + + let rpc_client = ReplicationLogClient::with_origin(channel.clone(), uri.clone()); + // TODO! setup replication + + let mut join_set = JoinSet::new(); + let namespace = name.clone(); + + let stats = make_stats( + &db_path, + &mut join_set, + db_config.clone(), + self.base.stats_sender.clone(), + name.clone(), + applied_frame_no_receiver.clone(), + ) + .await?; + + let connection_maker = MakeWriteProxyConn::new( + db_path.clone(), + self.base.extensions.clone(), + channel.clone(), + uri.clone(), + stats.clone(), + broadcaster, + db_config.clone(), + applied_frame_no_receiver, + self.base.max_response_size, + self.base.max_total_response_size, + primary_current_replication_index, + None, + resolve_attach_path, + self.make_wal_manager.clone(), + ) + .await? + .throttled( + self.base.max_concurrent_connections.clone(), + Some(DB_CREATE_TIMEOUT), + self.base.max_total_response_size, + self.base.max_concurrent_requests, + ); + + Ok(Namespace { + tasks: join_set, + db: Database::Replica(ReplicaDatabase { + connection_maker: Arc::new(connection_maker), + }), + name: name.clone(), + stats, + db_config_store: db_config, + path: db_path.into(), + }) + }) + } + + fn cleanup<'a>( + &'a self, + namespace: &'a NamespaceName, + _db_config: &DatabaseConfig, + _prune_all: bool, + _bottomless_db_id_init: NamespaceBottomlessDbIdInit, + ) -> Pin> + Send + 'a>> { + Box::pin(async move { + let ns_path = self.base.base_path.join("dbs").join(namespace.as_str()); + if ns_path.try_exists()? { + tracing::debug!("removing database directory: {}", ns_path.display()); + tokio::fs::remove_dir_all(ns_path).await?; + } + Ok(()) + }) + } + + fn fork<'a>( + &'a self, + _from_ns: &'a Namespace, + _from_config: MetaStoreHandle, + _to_ns: NamespaceName, + _to_config: MetaStoreHandle, + _timestamp: Option, + _store: NamespaceStore, + ) -> Pin> + Send + 'a>> { + Box::pin(std::future::ready(Err(crate::Error::Fork( + super::fork::ForkError::ForkReplica, + )))) + } +} From ded5ba7f859f6f6b94f1a1b6614e31521a591f01 Mon Sep 17 00:00:00 2001 From: ad hoc Date: Tue, 6 Aug 2024 16:54:14 +0200 Subject: [PATCH 11/13] fmt + remove dbgs --- libsql-server/src/http/admin/stats.rs | 2 - libsql-server/src/lib.rs | 2 - .../src/namespace/configurator/fork.rs | 7 ++- .../src/namespace/configurator/helpers.rs | 60 +++++++++--------- .../configurator/libsql_wal_replica.rs | 18 +++--- .../src/namespace/configurator/mod.rs | 18 ++++-- .../src/namespace/configurator/primary.rs | 13 ++-- .../src/namespace/configurator/schema.rs | 29 ++++++--- libsql-server/src/namespace/mod.rs | 2 +- libsql-server/src/namespace/store.rs | 13 +--- libsql-server/src/schema/scheduler.rs | 63 ++++++------------- libsql-server/tests/cluster/mod.rs | 29 +++------ 12 files changed, 111 insertions(+), 145 deletions(-) diff --git a/libsql-server/src/http/admin/stats.rs b/libsql-server/src/http/admin/stats.rs index 5fce92ba0a..f2948d4d7b 100644 --- a/libsql-server/src/http/admin/stats.rs +++ b/libsql-server/src/http/admin/stats.rs @@ -140,12 +140,10 @@ pub(super) async fn handle_stats( State(app_state): State>>, Path(namespace): Path, ) -> crate::Result> { - dbg!(); let stats = app_state .namespaces .stats(NamespaceName::from_string(namespace)?) .await?; - dbg!(); let resp: StatsResponse = stats.as_ref().into(); Ok(Json(resp)) diff --git a/libsql-server/src/lib.rs b/libsql-server/src/lib.rs index f5788dcebb..d26921dd00 100644 --- a/libsql-server/src/lib.rs +++ b/libsql-server/src/lib.rs @@ -591,7 +591,6 @@ where .configure(&mut join_set); } DatabaseKind::Replica => { - dbg!(); let (channel, uri) = client_config.clone().unwrap(); let replication_svc = ReplicationLogProxyService::new(channel.clone(), uri.clone()); let proxy_svc = ReplicaProxyService::new( @@ -611,7 +610,6 @@ where service_shutdown.clone(), ) .configure(&mut join_set); - dbg!(); } }; diff --git a/libsql-server/src/namespace/configurator/fork.rs b/libsql-server/src/namespace/configurator/fork.rs index 26a0b99b61..03f2ac03d8 100644 --- a/libsql-server/src/namespace/configurator/fork.rs +++ b/libsql-server/src/namespace/configurator/fork.rs @@ -58,7 +58,7 @@ pub(super) async fn fork( Database::Schema(db) => db.wal_wrapper.wrapper().logger(), _ => { return Err(crate::Error::Fork(ForkError::Internal(anyhow::Error::msg( - "Invalid source database type for fork", + "Invalid source database type for fork", )))); } }; @@ -114,7 +114,7 @@ pub struct ForkTask { pub to_namespace: NamespaceName, pub to_config: MetaStoreHandle, pub restore_to: Option, - pub store: NamespaceStore + pub store: NamespaceStore, } pub struct PointInTimeRestore { @@ -156,7 +156,8 @@ impl ForkTask { let dest_path = self.base_path.join("dbs").join(self.to_namespace.as_str()); tokio::fs::rename(temp_dir.path(), dest_path).await?; - self.store.make_namespace(&self.to_namespace, self.to_config, RestoreOption::Latest) + self.store + .make_namespace(&self.to_namespace, self.to_config, RestoreOption::Latest) .await .map_err(|e| ForkError::CreateNamespace(Box::new(e))) } diff --git a/libsql-server/src/namespace/configurator/helpers.rs b/libsql-server/src/namespace/configurator/helpers.rs index f43fa8a192..a5a4c5121d 100644 --- a/libsql-server/src/namespace/configurator/helpers.rs +++ b/libsql-server/src/namespace/configurator/helpers.rs @@ -6,26 +6,29 @@ use std::time::Duration; use anyhow::Context as _; use bottomless::replicator::Options; use bytes::Bytes; +use enclose::enclose; use futures::Stream; use libsql_sys::wal::Sqlite3WalManager; use tokio::io::AsyncBufReadExt as _; use tokio::sync::watch; use tokio::task::JoinSet; use tokio_util::io::StreamReader; -use enclose::enclose; use crate::connection::config::DatabaseConfig; use crate::connection::connection_manager::InnerWalManager; use crate::connection::libsql::{open_conn, MakeLibSqlConn}; use crate::connection::{Connection as _, MakeConnection as _}; +use crate::database::{PrimaryConnection, PrimaryConnectionMaker}; use crate::error::LoadDumpError; +use crate::namespace::broadcasters::BroadcasterHandle; +use crate::namespace::meta_store::MetaStoreHandle; +use crate::namespace::replication_wal::{make_replication_wal_wrapper, ReplicationWalWrapper}; +use crate::namespace::{ + NamespaceBottomlessDbId, NamespaceBottomlessDbIdInit, NamespaceName, ResolveNamespacePathFn, + RestoreOption, +}; use crate::replication::{FrameNo, ReplicationLogger}; use crate::stats::Stats; -use crate::namespace::{NamespaceBottomlessDbId, NamespaceBottomlessDbIdInit, NamespaceName, ResolveNamespacePathFn, RestoreOption}; -use crate::namespace::replication_wal::{make_replication_wal_wrapper, ReplicationWalWrapper}; -use crate::namespace::meta_store::MetaStoreHandle; -use crate::namespace::broadcasters::BroadcasterHandle; -use crate::database::{PrimaryConnection, PrimaryConnectionMaker}; use crate::{StatsSender, BLOCKING_RT, DB_CREATE_TIMEOUT, DEFAULT_AUTO_CHECKPOINT}; use super::{BaseNamespaceConfig, PrimaryExtraConfig}; @@ -74,8 +77,7 @@ pub(super) async fn make_primary_connection_maker( tracing::debug!("Checkpointed before initializing bottomless"); let options = make_bottomless_options(options, bottomless_db_id, name.clone()); let (replicator, did_recover) = - init_bottomless_replicator(db_path.join("data"), options, &restore_option) - .await?; + init_bottomless_replicator(db_path.join("data"), options, &restore_option).await?; tracing::debug!("Completed init of bottomless replicator"); is_dirty |= did_recover; Some(replicator) @@ -93,14 +95,14 @@ pub(super) async fn make_primary_connection_maker( }; let logger = Arc::new(ReplicationLogger::open( - &db_path, - primary_config.max_log_size, - primary_config.max_log_duration, - is_dirty, - auto_checkpoint, - primary_config.scripted_backup.clone(), - name.clone(), - None, + &db_path, + primary_config.max_log_size, + primary_config.max_log_duration, + is_dirty, + auto_checkpoint, + primary_config.scripted_backup.clone(), + name.clone(), + None, )?); tracing::debug!("sending stats"); @@ -113,7 +115,7 @@ pub(super) async fn make_primary_connection_maker( name.clone(), logger.new_frame_notifier.subscribe(), ) - .await?; + .await?; tracing::debug!("Making replication wal wrapper"); let wal_wrapper = make_replication_wal_wrapper(bottomless_replicator, logger.clone()); @@ -136,13 +138,13 @@ pub(super) async fn make_primary_connection_maker( resolve_attach_path, make_wal_manager.clone(), ) - .await? - .throttled( - base_config.max_concurrent_connections.clone(), - Some(DB_CREATE_TIMEOUT), - base_config.max_total_response_size, - base_config.max_concurrent_requests, - ); + .await? + .throttled( + base_config.max_concurrent_connections.clone(), + Some(DB_CREATE_TIMEOUT), + base_config.max_total_response_size, + base_config.max_concurrent_requests, + ); tracing::debug!("Completed opening libsql connection"); @@ -356,10 +358,7 @@ pub(super) async fn make_stats( } }); - join_set.spawn(run_storage_monitor( - db_path.into(), - Arc::downgrade(&stats), - )); + join_set.spawn(run_storage_monitor(db_path.into(), Arc::downgrade(&stats))); tracing::debug!("done sending stats, and creating bg tasks"); @@ -369,10 +368,7 @@ pub(super) async fn make_stats( // Periodically check the storage used by the database and save it in the Stats structure. // TODO: Once we have a separate fiber that does WAL checkpoints, running this routine // right after checkpointing is exactly where it should be done. -async fn run_storage_monitor( - db_path: PathBuf, - stats: Weak, -) -> anyhow::Result<()> { +async fn run_storage_monitor(db_path: PathBuf, stats: Weak) -> anyhow::Result<()> { // on initialization, the database file doesn't exist yet, so we wait a bit for it to be // created tokio::time::sleep(Duration::from_secs(1)).await; diff --git a/libsql-server/src/namespace/configurator/libsql_wal_replica.rs b/libsql-server/src/namespace/configurator/libsql_wal_replica.rs index 6b2519cf33..f26738ec2a 100644 --- a/libsql-server/src/namespace/configurator/libsql_wal_replica.rs +++ b/libsql-server/src/namespace/configurator/libsql_wal_replica.rs @@ -1,5 +1,5 @@ -use std::pin::Pin; use std::future::Future; +use std::pin::Pin; use std::sync::Arc; use chrono::prelude::NaiveDateTime; @@ -66,7 +66,7 @@ impl ConfigureNamespace for LibsqlWalReplicaConfigurator { name.clone(), applied_frame_no_receiver.clone(), ) - .await?; + .await?; let connection_maker = MakeWriteProxyConn::new( db_path.clone(), @@ -84,13 +84,13 @@ impl ConfigureNamespace for LibsqlWalReplicaConfigurator { resolve_attach_path, self.make_wal_manager.clone(), ) - .await? - .throttled( - self.base.max_concurrent_connections.clone(), - Some(DB_CREATE_TIMEOUT), - self.base.max_total_response_size, - self.base.max_concurrent_requests, - ); + .await? + .throttled( + self.base.max_concurrent_connections.clone(), + Some(DB_CREATE_TIMEOUT), + self.base.max_total_response_size, + self.base.max_concurrent_requests, + ); Ok(Namespace { tasks: join_set, diff --git a/libsql-server/src/namespace/configurator/mod.rs b/libsql-server/src/namespace/configurator/mod.rs index e5db335ff6..9122fc18de 100644 --- a/libsql-server/src/namespace/configurator/mod.rs +++ b/libsql-server/src/namespace/configurator/mod.rs @@ -13,13 +13,17 @@ use crate::StatsSender; use super::broadcasters::BroadcasterHandle; use super::meta_store::MetaStoreHandle; -use super::{Namespace, NamespaceBottomlessDbIdInit, NamespaceName, NamespaceStore, ResetCb, ResolveNamespacePathFn, RestoreOption}; +use super::{ + Namespace, NamespaceBottomlessDbIdInit, NamespaceName, NamespaceStore, ResetCb, + ResolveNamespacePathFn, RestoreOption, +}; +pub mod fork; mod helpers; +mod libsql_wal_replica; mod primary; mod replica; mod schema; -pub mod fork; pub use primary::PrimaryConfigurator; pub use replica::ReplicaConfigurator; @@ -68,12 +72,18 @@ impl NamespaceConfigurators { } } - pub fn with_primary(&mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> &mut Self { + pub fn with_primary( + &mut self, + c: impl ConfigureNamespace + Send + Sync + 'static, + ) -> &mut Self { self.primary_configurator = Some(Box::new(c)); self } - pub fn with_replica(&mut self, c: impl ConfigureNamespace + Send + Sync + 'static) -> &mut Self { + pub fn with_replica( + &mut self, + c: impl ConfigureNamespace + Send + Sync + 'static, + ) -> &mut Self { self.replica_configurator = Some(Box::new(c)); self } diff --git a/libsql-server/src/namespace/configurator/primary.rs b/libsql-server/src/namespace/configurator/primary.rs index 4351f6a3ac..6c245a6e8f 100644 --- a/libsql-server/src/namespace/configurator/primary.rs +++ b/libsql-server/src/namespace/configurator/primary.rs @@ -12,8 +12,8 @@ use crate::namespace::broadcasters::BroadcasterHandle; use crate::namespace::configurator::helpers::make_primary_connection_maker; use crate::namespace::meta_store::MetaStoreHandle; use crate::namespace::{ - Namespace, NamespaceBottomlessDbIdInit, NamespaceName, NamespaceStore, - ResetCb, ResolveNamespacePathFn, RestoreOption, + Namespace, NamespaceBottomlessDbIdInit, NamespaceName, NamespaceStore, ResetCb, + ResolveNamespacePathFn, RestoreOption, }; use crate::run_periodic_checkpoint; use crate::schema::{has_pending_migration_task, setup_migration_table}; @@ -168,7 +168,8 @@ impl ConfigureNamespace for PrimaryConfigurator { db_config, prune_all, bottomless_db_id_init, - ).await + ) + .await }) } @@ -186,10 +187,10 @@ impl ConfigureNamespace for PrimaryConfigurator { from_config, to_ns, to_config, - timestamp, + timestamp, store, &self.primary_config, - self.base.base_path.clone())) + self.base.base_path.clone(), + )) } } - diff --git a/libsql-server/src/namespace/configurator/schema.rs b/libsql-server/src/namespace/configurator/schema.rs index e55c706fec..98e679513a 100644 --- a/libsql-server/src/namespace/configurator/schema.rs +++ b/libsql-server/src/namespace/configurator/schema.rs @@ -6,12 +6,11 @@ use tokio::task::JoinSet; use crate::connection::config::DatabaseConfig; use crate::connection::connection_manager::InnerWalManager; use crate::database::{Database, SchemaDatabase}; +use crate::namespace::broadcasters::BroadcasterHandle; use crate::namespace::meta_store::MetaStoreHandle; use crate::namespace::{ - Namespace, NamespaceName, NamespaceStore, - ResetCb, ResolveNamespacePathFn, RestoreOption, + Namespace, NamespaceName, NamespaceStore, ResetCb, ResolveNamespacePathFn, RestoreOption, }; -use crate::namespace::broadcasters::BroadcasterHandle; use crate::schema::SchedulerHandle; use super::helpers::{cleanup_primary, make_primary_connection_maker}; @@ -25,8 +24,18 @@ pub struct SchemaConfigurator { } impl SchemaConfigurator { - pub fn new(base: BaseNamespaceConfig, primary_config: PrimaryExtraConfig, make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, migration_scheduler: SchedulerHandle) -> Self { - Self { base, primary_config, make_wal_manager, migration_scheduler } + pub fn new( + base: BaseNamespaceConfig, + primary_config: PrimaryExtraConfig, + make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, + migration_scheduler: SchedulerHandle, + ) -> Self { + Self { + base, + primary_config, + make_wal_manager, + migration_scheduler, + } } } @@ -58,7 +67,7 @@ impl ConfigureNamespace for SchemaConfigurator { &mut join_set, resolve_attach_path, broadcaster, - self.make_wal_manager.clone() + self.make_wal_manager.clone(), ) .await?; @@ -94,7 +103,8 @@ impl ConfigureNamespace for SchemaConfigurator { db_config, prune_all, bottomless_db_id_init, - ).await + ) + .await }) } @@ -112,9 +122,10 @@ impl ConfigureNamespace for SchemaConfigurator { from_config, to_ns, to_config, - timestamp, + timestamp, store, &self.primary_config, - self.base.base_path.clone())) + self.base.base_path.clone(), + )) } } diff --git a/libsql-server/src/namespace/mod.rs b/libsql-server/src/namespace/mod.rs index 7cfa6b351c..2a2e3eb211 100644 --- a/libsql-server/src/namespace/mod.rs +++ b/libsql-server/src/namespace/mod.rs @@ -19,12 +19,12 @@ pub use self::name::NamespaceName; pub use self::store::NamespaceStore; pub mod broadcasters; +pub(crate) mod configurator; pub mod meta_store; mod name; pub mod replication_wal; mod schema_lock; mod store; -pub(crate) mod configurator; pub type ResetCb = Box; pub type ResolveNamespacePathFn = diff --git a/libsql-server/src/namespace/store.rs b/libsql-server/src/namespace/store.rs index b2b5d33032..a78e4f59b0 100644 --- a/libsql-server/src/namespace/store.rs +++ b/libsql-server/src/namespace/store.rs @@ -327,7 +327,6 @@ impl NamespaceStore { where Fun: FnOnce(&Namespace) -> R, { - dbg!(); if namespace != NamespaceName::default() && !self.inner.metadata.exists(&namespace) && !self.inner.allow_lazy_creation @@ -335,7 +334,6 @@ impl NamespaceStore { return Err(Error::NamespaceDoesntExist(namespace.to_string())); } - dbg!(); let f = { let name = namespace.clone(); move |ns: NamespaceEntry| async move { @@ -348,9 +346,7 @@ impl NamespaceStore { } }; - dbg!(); let handle = self.inner.metadata.handle(namespace.to_owned()); - dbg!(); f(self .load_namespace(&namespace, handle, RestoreOption::Latest) .await?) @@ -377,7 +373,6 @@ impl NamespaceStore { config: MetaStoreHandle, restore_option: RestoreOption, ) -> crate::Result { - dbg!(); let ns = self .get_configurator(&config.get()) .setup( @@ -391,7 +386,6 @@ impl NamespaceStore { ) .await?; - dbg!(); Ok(ns) } @@ -401,17 +395,13 @@ impl NamespaceStore { db_config: MetaStoreHandle, restore_option: RestoreOption, ) -> crate::Result { - dbg!(); let init = async { - dbg!(); let ns = self .make_namespace(namespace, db_config, restore_option) .await?; - dbg!(); Ok(Some(ns)) }; - dbg!(); let before_load = Instant::now(); let ns = self .inner @@ -420,8 +410,7 @@ impl NamespaceStore { namespace.clone(), init.map_ok(|ns| Arc::new(RwLock::new(ns))), ) - .await.map_err(|e| dbg!(e))?; - dbg!(); + .await?; NAMESPACE_LOAD_LATENCY.record(before_load.elapsed()); Ok(ns) diff --git a/libsql-server/src/schema/scheduler.rs b/libsql-server/src/schema/scheduler.rs index a8195cbbd0..57916bb9a5 100644 --- a/libsql-server/src/schema/scheduler.rs +++ b/libsql-server/src/schema/scheduler.rs @@ -830,16 +830,10 @@ mod test { .unwrap(); let (sender, mut receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new( - false, - false, - 10, - meta_store, - config, - DatabaseKind::Primary - ) - .await - .unwrap(); + let store = + NamespaceStore::new(false, false, 10, meta_store, config, DatabaseKind::Primary) + .await + .unwrap(); let mut scheduler = Scheduler::new(store.clone(), maker().unwrap()) .await .unwrap(); @@ -961,16 +955,10 @@ mod test { .unwrap(); let (sender, mut receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new( - false, - false, - 10, - meta_store, - config, - DatabaseKind::Primary - ) - .await - .unwrap(); + let store = + NamespaceStore::new(false, false, 10, meta_store, config, DatabaseKind::Primary) + .await + .unwrap(); let mut scheduler = Scheduler::new(store.clone(), maker().unwrap()) .await .unwrap(); @@ -1044,16 +1032,10 @@ mod test { .unwrap(); let (sender, _receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new( - false, - false, - 10, - meta_store, - config, - DatabaseKind::Primary, - ) - .await - .unwrap(); + let store = + NamespaceStore::new(false, false, 10, meta_store, config, DatabaseKind::Primary) + .await + .unwrap(); store .with("ns".into(), |ns| { @@ -1078,9 +1060,10 @@ mod test { .unwrap(); let (sender, mut receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new(false, false, 10, meta_store, config, DatabaseKind::Primary) - .await - .unwrap(); + let store = + NamespaceStore::new(false, false, 10, meta_store, config, DatabaseKind::Primary) + .await + .unwrap(); let mut scheduler = Scheduler::new(store.clone(), maker().unwrap()) .await .unwrap(); @@ -1151,16 +1134,10 @@ mod test { .unwrap(); let (sender, _receiver) = mpsc::channel(100); let config = make_config(sender.clone().into(), tmp.path()); - let store = NamespaceStore::new( - false, - false, - 10, - meta_store, - config, - DatabaseKind::Primary - ) - .await - .unwrap(); + let store = + NamespaceStore::new(false, false, 10, meta_store, config, DatabaseKind::Primary) + .await + .unwrap(); let scheduler = Scheduler::new(store.clone(), maker().unwrap()) .await .unwrap(); diff --git a/libsql-server/tests/cluster/mod.rs b/libsql-server/tests/cluster/mod.rs index 8f214bd05e..1171d4a5d0 100644 --- a/libsql-server/tests/cluster/mod.rs +++ b/libsql-server/tests/cluster/mod.rs @@ -149,29 +149,23 @@ fn sync_many_replica() { let mut sim = Builder::new() .simulation_duration(Duration::from_secs(1000)) .build(); - dbg!(); make_cluster(&mut sim, NUM_REPLICA, true); - dbg!(); sim.client("client", async { let db = Database::open_remote_with_connector("http://primary:8080", "", TurmoilConnector)?; let conn = db.connect()?; - dbg!(); conn.execute("create table test (x)", ()).await?; - dbg!(); conn.execute("insert into test values (42)", ()).await?; - dbg!(); async fn get_frame_no(url: &str) -> Option { let client = Client::new(); - dbg!(); Some( - dbg!(client - .get(url) - .await - .unwrap() - .json::() - .await) + client + .get(url) + .await + .unwrap() + .json::() + .await .unwrap() .get("replication_index")? .as_u64() @@ -179,7 +173,6 @@ fn sync_many_replica() { ) } - dbg!(); let primary_fno = loop { if let Some(fno) = get_frame_no("http://primary:9090/v1/namespaces/default/stats").await { @@ -187,15 +180,13 @@ fn sync_many_replica() { } }; - dbg!(); // wait for all replicas to sync let mut join_set = JoinSet::new(); for i in 0..NUM_REPLICA { join_set.spawn(async move { let uri = format!("http://replica{i}:9090/v1/namespaces/default/stats"); - dbg!(); loop { - if let Some(replica_fno) = dbg!(get_frame_no(&uri).await) { + if let Some(replica_fno) = get_frame_no(&uri).await { if replica_fno == primary_fno { break; } @@ -205,10 +196,8 @@ fn sync_many_replica() { }); } - dbg!(); while join_set.join_next().await.is_some() {} - dbg!(); for i in 0..NUM_REPLICA { let db = Database::open_remote_with_connector( format!("http://replica{i}:8080"), @@ -223,10 +212,8 @@ fn sync_many_replica() { )); } - dbg!(); let client = Client::new(); - dbg!(); let stats = client .get("http://primary:9090/v1/namespaces/default/stats") .await? @@ -234,14 +221,12 @@ fn sync_many_replica() { .await .unwrap(); - dbg!(); let stat = stats .get("embedded_replica_frames_replicated") .unwrap() .as_u64() .unwrap(); - dbg!(); assert_eq!(stat, 0); Ok(()) From e5b8c31005069982de27ad0b58b109929f45bf4b Mon Sep 17 00:00:00 2001 From: ad hoc Date: Tue, 6 Aug 2024 17:01:34 +0200 Subject: [PATCH 12/13] comment out libsql-wal replica configurator --- libsql-server/src/lib.rs | 34 +++--- .../configurator/libsql_wal_replica.rs | 115 +++++++++--------- .../src/namespace/configurator/mod.rs | 2 +- 3 files changed, 79 insertions(+), 72 deletions(-) diff --git a/libsql-server/src/lib.rs b/libsql-server/src/lib.rs index d26921dd00..9bf0419932 100644 --- a/libsql-server/src/lib.rs +++ b/libsql-server/src/lib.rs @@ -745,21 +745,27 @@ where }); let make_wal_manager = Arc::new(move || EitherWAL::B(wal.clone())); - let mut configurators = NamespaceConfigurators::empty(); + // let mut configurators = NamespaceConfigurators::empty(); + + // match client_config { + // Some(_) => todo!("configure replica"), + // // configure primary + // None => self.configure_primary_common( + // base_config, + // &mut configurators, + // make_wal_manager, + // migration_scheduler_handle, + // scripted_backup, + // ), + // } - match client_config { - Some(_) => todo!("configure replica"), - // configure primary - None => self.configure_primary_common( - base_config, - &mut configurators, - make_wal_manager, - migration_scheduler_handle, - scripted_backup, - ), - } - - Ok(configurators) + self.configurators_common( + base_config, + client_config, + make_wal_manager, + migration_scheduler_handle, + scripted_backup, + ) } #[cfg(feature = "durable-wal")] diff --git a/libsql-server/src/namespace/configurator/libsql_wal_replica.rs b/libsql-server/src/namespace/configurator/libsql_wal_replica.rs index f26738ec2a..6ab6cc52ef 100644 --- a/libsql-server/src/namespace/configurator/libsql_wal_replica.rs +++ b/libsql-server/src/namespace/configurator/libsql_wal_replica.rs @@ -46,63 +46,64 @@ impl ConfigureNamespace for LibsqlWalReplicaConfigurator { store: NamespaceStore, broadcaster: BroadcasterHandle, ) -> Pin> + Send + 'a>> { - Box::pin(async move { - tracing::debug!("creating replica namespace"); - let db_path = self.base.base_path.join("dbs").join(name.as_str()); - let channel = self.channel.clone(); - let uri = self.uri.clone(); - - let rpc_client = ReplicationLogClient::with_origin(channel.clone(), uri.clone()); - // TODO! setup replication - - let mut join_set = JoinSet::new(); - let namespace = name.clone(); - - let stats = make_stats( - &db_path, - &mut join_set, - db_config.clone(), - self.base.stats_sender.clone(), - name.clone(), - applied_frame_no_receiver.clone(), - ) - .await?; - - let connection_maker = MakeWriteProxyConn::new( - db_path.clone(), - self.base.extensions.clone(), - channel.clone(), - uri.clone(), - stats.clone(), - broadcaster, - db_config.clone(), - applied_frame_no_receiver, - self.base.max_response_size, - self.base.max_total_response_size, - primary_current_replication_index, - None, - resolve_attach_path, - self.make_wal_manager.clone(), - ) - .await? - .throttled( - self.base.max_concurrent_connections.clone(), - Some(DB_CREATE_TIMEOUT), - self.base.max_total_response_size, - self.base.max_concurrent_requests, - ); - - Ok(Namespace { - tasks: join_set, - db: Database::Replica(ReplicaDatabase { - connection_maker: Arc::new(connection_maker), - }), - name: name.clone(), - stats, - db_config_store: db_config, - path: db_path.into(), - }) - }) + todo!() + // Box::pin(async move { + // tracing::debug!("creating replica namespace"); + // let db_path = self.base.base_path.join("dbs").join(name.as_str()); + // let channel = self.channel.clone(); + // let uri = self.uri.clone(); + // + // let rpc_client = ReplicationLogClient::with_origin(channel.clone(), uri.clone()); + // // TODO! setup replication + // + // let mut join_set = JoinSet::new(); + // let namespace = name.clone(); + // + // let stats = make_stats( + // &db_path, + // &mut join_set, + // db_config.clone(), + // self.base.stats_sender.clone(), + // name.clone(), + // applied_frame_no_receiver.clone(), + // ) + // .await?; + // + // let connection_maker = MakeWriteProxyConn::new( + // db_path.clone(), + // self.base.extensions.clone(), + // channel.clone(), + // uri.clone(), + // stats.clone(), + // broadcaster, + // db_config.clone(), + // applied_frame_no_receiver, + // self.base.max_response_size, + // self.base.max_total_response_size, + // primary_current_replication_index, + // None, + // resolve_attach_path, + // self.make_wal_manager.clone(), + // ) + // .await? + // .throttled( + // self.base.max_concurrent_connections.clone(), + // Some(DB_CREATE_TIMEOUT), + // self.base.max_total_response_size, + // self.base.max_concurrent_requests, + // ); + // + // Ok(Namespace { + // tasks: join_set, + // db: Database::Replica(ReplicaDatabase { + // connection_maker: Arc::new(connection_maker), + // }), + // name: name.clone(), + // stats, + // db_config_store: db_config, + // path: db_path.into(), + // }) + // }) } fn cleanup<'a>( diff --git a/libsql-server/src/namespace/configurator/mod.rs b/libsql-server/src/namespace/configurator/mod.rs index 9122fc18de..0f8dcbd481 100644 --- a/libsql-server/src/namespace/configurator/mod.rs +++ b/libsql-server/src/namespace/configurator/mod.rs @@ -20,7 +20,7 @@ use super::{ pub mod fork; mod helpers; -mod libsql_wal_replica; +// mod libsql_wal_replica; mod primary; mod replica; mod schema; From 6e7fb9f06a901fe20a340cf9e54b1523c44f8eb5 Mon Sep 17 00:00:00 2001 From: ad hoc Date: Tue, 6 Aug 2024 18:24:20 +0200 Subject: [PATCH 13/13] restore encryption config we don't actually care, but let's do it for completeness --- libsql-server/src/lib.rs | 1 + .../src/namespace/configurator/helpers.rs | 23 +++++++++++++++---- .../src/namespace/configurator/mod.rs | 2 ++ .../src/namespace/configurator/primary.rs | 8 ++++++- .../src/namespace/configurator/replica.rs | 1 + .../src/namespace/configurator/schema.rs | 1 + libsql-server/src/schema/scheduler.rs | 1 + 7 files changed, 31 insertions(+), 6 deletions(-) diff --git a/libsql-server/src/lib.rs b/libsql-server/src/lib.rs index 9bf0419932..4b97b442f5 100644 --- a/libsql-server/src/lib.rs +++ b/libsql-server/src/lib.rs @@ -458,6 +458,7 @@ where max_total_response_size: self.db_config.max_total_response_size, max_concurrent_connections: Arc::new(Semaphore::new(self.max_concurrent_connections)), max_concurrent_requests: self.db_config.max_concurrent_requests, + encryption_config: self.db_config.encryption_config.clone(), }; let configurators = self diff --git a/libsql-server/src/namespace/configurator/helpers.rs b/libsql-server/src/namespace/configurator/helpers.rs index a5a4c5121d..355b1b1472 100644 --- a/libsql-server/src/namespace/configurator/helpers.rs +++ b/libsql-server/src/namespace/configurator/helpers.rs @@ -9,6 +9,7 @@ use bytes::Bytes; use enclose::enclose; use futures::Stream; use libsql_sys::wal::Sqlite3WalManager; +use libsql_sys::EncryptionConfig; use tokio::io::AsyncBufReadExt as _; use tokio::sync::watch; use tokio::task::JoinSet; @@ -49,6 +50,7 @@ pub(super) async fn make_primary_connection_maker( resolve_attach_path: ResolveNamespacePathFn, broadcaster: BroadcasterHandle, make_wal_manager: Arc InnerWalManager + Sync + Send + 'static>, + encryption_config: Option, ) -> crate::Result<(PrimaryConnectionMaker, ReplicationWalWrapper, Arc)> { let db_config = meta_store_handle.get(); let bottomless_db_id = NamespaceBottomlessDbId::from_config(&db_config); @@ -102,7 +104,7 @@ pub(super) async fn make_primary_connection_maker( auto_checkpoint, primary_config.scripted_backup.clone(), name.clone(), - None, + encryption_config.clone(), )?); tracing::debug!("sending stats"); @@ -114,6 +116,7 @@ pub(super) async fn make_primary_connection_maker( base_config.stats_sender.clone(), name.clone(), logger.new_frame_notifier.subscribe(), + base_config.encryption_config.clone(), ) .await?; @@ -133,7 +136,7 @@ pub(super) async fn make_primary_connection_maker( base_config.max_total_response_size, auto_checkpoint, logger.new_frame_notifier.subscribe(), - None, + encryption_config, block_writes, resolve_attach_path, make_wal_manager.clone(), @@ -332,6 +335,7 @@ pub(super) async fn make_stats( stats_sender: StatsSender, name: NamespaceName, mut current_frame_no: watch::Receiver>, + encryption_config: Option, ) -> anyhow::Result> { tracing::debug!("creating stats type"); let stats = Stats::new(name.clone(), db_path, join_set).await?; @@ -358,7 +362,11 @@ pub(super) async fn make_stats( } }); - join_set.spawn(run_storage_monitor(db_path.into(), Arc::downgrade(&stats))); + join_set.spawn(run_storage_monitor( + db_path.into(), + Arc::downgrade(&stats), + encryption_config, + )); tracing::debug!("done sending stats, and creating bg tasks"); @@ -368,7 +376,11 @@ pub(super) async fn make_stats( // Periodically check the storage used by the database and save it in the Stats structure. // TODO: Once we have a separate fiber that does WAL checkpoints, running this routine // right after checkpointing is exactly where it should be done. -async fn run_storage_monitor(db_path: PathBuf, stats: Weak) -> anyhow::Result<()> { +async fn run_storage_monitor( + db_path: PathBuf, + stats: Weak, + encryption_config: Option, +) -> anyhow::Result<()> { // on initialization, the database file doesn't exist yet, so we wait a bit for it to be // created tokio::time::sleep(Duration::from_secs(1)).await; @@ -381,11 +393,12 @@ async fn run_storage_monitor(db_path: PathBuf, stats: Weak) -> anyhow::Re return Ok(()); }; + let encryption_config = encryption_config.clone(); let _ = tokio::task::spawn_blocking(move || { // because closing the last connection interferes with opening a new one, we lazily // initialize a connection here, and keep it alive for the entirety of the program. If we // fail to open it, we wait for `duration` and try again later. - match open_conn(&db_path, Sqlite3WalManager::new(), Some(rusqlite::OpenFlags::SQLITE_OPEN_READ_ONLY), None) { + match open_conn(&db_path, Sqlite3WalManager::new(), Some(rusqlite::OpenFlags::SQLITE_OPEN_READ_ONLY), encryption_config) { Ok(mut conn) => { if let Ok(tx) = conn.transaction() { let page_count = tx.query_row("pragma page_count;", [], |row| { row.get::(0) }); diff --git a/libsql-server/src/namespace/configurator/mod.rs b/libsql-server/src/namespace/configurator/mod.rs index 0f8dcbd481..b96d5a3824 100644 --- a/libsql-server/src/namespace/configurator/mod.rs +++ b/libsql-server/src/namespace/configurator/mod.rs @@ -5,6 +5,7 @@ use std::time::Duration; use chrono::NaiveDateTime; use futures::Future; +use libsql_sys::EncryptionConfig; use tokio::sync::Semaphore; use crate::connection::config::DatabaseConfig; @@ -38,6 +39,7 @@ pub struct BaseNamespaceConfig { pub(crate) max_total_response_size: u64, pub(crate) max_concurrent_connections: Arc, pub(crate) max_concurrent_requests: u64, + pub(crate) encryption_config: Option, } #[derive(Clone)] diff --git a/libsql-server/src/namespace/configurator/primary.rs b/libsql-server/src/namespace/configurator/primary.rs index 6c245a6e8f..03cdd2fd7b 100644 --- a/libsql-server/src/namespace/configurator/primary.rs +++ b/libsql-server/src/namespace/configurator/primary.rs @@ -1,7 +1,10 @@ +use std::path::Path; +use std::pin::Pin; use std::sync::atomic::{AtomicBool, Ordering}; -use std::{path::Path, pin::Pin, sync::Arc}; +use std::sync::Arc; use futures::prelude::Future; +use libsql_sys::EncryptionConfig; use tokio::task::JoinSet; use crate::connection::config::DatabaseConfig; @@ -49,6 +52,7 @@ impl PrimaryConfigurator { resolve_attach_path: ResolveNamespacePathFn, db_path: Arc, broadcaster: BroadcasterHandle, + encryption_config: Option, ) -> crate::Result { let mut join_set = JoinSet::new(); @@ -67,6 +71,7 @@ impl PrimaryConfigurator { resolve_attach_path, broadcaster, self.make_wal_manager.clone(), + encryption_config, ) .await?; let connection_maker = Arc::new(connection_maker); @@ -135,6 +140,7 @@ impl ConfigureNamespace for PrimaryConfigurator { resolve_attach_path, db_path.clone(), broadcaster, + self.base.encryption_config.clone(), ) .await { diff --git a/libsql-server/src/namespace/configurator/replica.rs b/libsql-server/src/namespace/configurator/replica.rs index 61dd48b0bf..84ebadb897 100644 --- a/libsql-server/src/namespace/configurator/replica.rs +++ b/libsql-server/src/namespace/configurator/replica.rs @@ -169,6 +169,7 @@ impl ConfigureNamespace for ReplicaConfigurator { self.base.stats_sender.clone(), name.clone(), applied_frame_no_receiver.clone(), + self.base.encryption_config.clone(), ) .await?; diff --git a/libsql-server/src/namespace/configurator/schema.rs b/libsql-server/src/namespace/configurator/schema.rs index 98e679513a..f95c8abf51 100644 --- a/libsql-server/src/namespace/configurator/schema.rs +++ b/libsql-server/src/namespace/configurator/schema.rs @@ -68,6 +68,7 @@ impl ConfigureNamespace for SchemaConfigurator { resolve_attach_path, broadcaster, self.make_wal_manager.clone(), + self.base.encryption_config.clone(), ) .await?; diff --git a/libsql-server/src/schema/scheduler.rs b/libsql-server/src/schema/scheduler.rs index 57916bb9a5..01a3d795d8 100644 --- a/libsql-server/src/schema/scheduler.rs +++ b/libsql-server/src/schema/scheduler.rs @@ -917,6 +917,7 @@ mod test { max_total_response_size: 100000000000, max_concurrent_connections: Arc::new(Semaphore::new(10)), max_concurrent_requests: 10000, + encryption_config: None, }; let primary_config = PrimaryExtraConfig {