From 36cdd682f9510e1a559d0293bf9257c389aec687 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 7 Jul 2023 12:57:48 +0800 Subject: [PATCH 01/15] refactor `ShardTablesCache` to `ShardSet`, move all internal operations of a shard from `ShardSet` to `Shard`. --- catalog_impls/src/volatile.rs | 12 +- cluster/src/cluster_impl.rs | 143 +++------------ cluster/src/lib.rs | 13 +- cluster/src/shard_set.rs | 176 +++++++++++++++++++ cluster/src/shard_tables_cache.rs | 279 ------------------------------ src/setup.rs | 6 +- 6 files changed, 217 insertions(+), 412 deletions(-) create mode 100644 cluster/src/shard_set.rs delete mode 100644 cluster/src/shard_tables_cache.rs diff --git a/catalog_impls/src/volatile.rs b/catalog_impls/src/volatile.rs index dcf0ecd173..236418c022 100644 --- a/catalog_impls/src/volatile.rs +++ b/catalog_impls/src/volatile.rs @@ -20,7 +20,7 @@ use catalog::{ }, Catalog, CatalogRef, CreateSchemaWithCause, }; -use cluster::shard_tables_cache::ShardTablesCache; +use cluster::shard_set::ShardSet; use common_types::schema::SchemaName; use common_util::error::BoxError; use log::{debug, info}; @@ -32,12 +32,12 @@ use tokio::sync::Mutex; /// ManagerImpl manages multiple volatile catalogs. pub struct ManagerImpl { catalogs: HashMap>, - shard_tables_cache: ShardTablesCache, + shard_tables_cache: ShardSet, meta_client: MetaClientRef, } impl ManagerImpl { - pub fn new(shard_tables_cache: ShardTablesCache, meta_client: MetaClientRef) -> Self { + pub fn new(shard_tables_cache: ShardSet, meta_client: MetaClientRef) -> Self { let mut manager = ManagerImpl { catalogs: HashMap::new(), shard_tables_cache, @@ -107,7 +107,7 @@ struct CatalogImpl { name: String, /// All the schemas belonging to the catalog. schemas: RwLock>, - shard_tables_cache: ShardTablesCache, + shard_tables_cache: ShardSet, meta_client: MetaClientRef, } @@ -188,7 +188,7 @@ struct SchemaImpl { /// Schema name schema_name: String, schema_id: SchemaId, - shard_tables_cache: ShardTablesCache, + shard_tables_cache: ShardSet, /// Tables of schema tables: RwLock>, /// Guard for creating/dropping table @@ -200,7 +200,7 @@ impl SchemaImpl { catalog_name: String, schema_name: String, schema_id: SchemaId, - shard_tables_cache: ShardTablesCache, + shard_tables_cache: ShardSet, ) -> Self { Self { catalog_name, diff --git a/cluster/src/cluster_impl.rs b/cluster/src/cluster_impl.rs index ee88794d4f..a9bdbb7df1 100644 --- a/cluster/src/cluster_impl.rs +++ b/cluster/src/cluster_impl.rs @@ -36,7 +36,7 @@ use tokio::{ use crate::{ config::ClusterConfig, shard_lock_manager::{ShardLockManager, ShardLockManagerRef}, - shard_tables_cache::ShardTablesCache, + shard_set::{ShardRef, ShardSet}, topology::ClusterTopology, Cluster, ClusterNodesNotFound, ClusterNodesResp, EtcdClientFailureWithCause, Internal, InvalidArguments, MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, @@ -61,7 +61,7 @@ pub struct ClusterImpl { impl ClusterImpl { pub async fn try_new( node_name: String, - shard_tables_cache: ShardTablesCache, + shard_tables_cache: ShardSet, meta_client: MetaClientRef, config: ClusterConfig, runtime: Arc, @@ -110,7 +110,14 @@ impl ClusterImpl { let handle = self.runtime.spawn(async move { loop { - let shard_infos = inner.shard_tables_cache.all_shard_infos(); + let shards = inner.shard_set.all_shards(); + let mut shard_infos = Vec::with_capacity(shards.len()); + // TODO: required to acquire the read lock of each shard now, can we optimize + // this? + for shard in shards { + let shard = shard.read().await; + shard_infos.push(shard.shard_info.clone()); + } info!("Node heartbeat to meta, shard infos:{:?}", shard_infos); let resp = inner.meta_client.send_heartbeat(shard_infos).await; @@ -163,15 +170,15 @@ impl ClusterImpl { } struct Inner { - shard_tables_cache: ShardTablesCache, + shard_set: ShardSet, meta_client: MetaClientRef, topology: RwLock, } impl Inner { - fn new(shard_tables_cache: ShardTablesCache, meta_client: MetaClientRef) -> Result { + fn new(shard_tables_cache: ShardSet, meta_client: MetaClientRef) -> Result { Ok(Self { - shard_tables_cache, + shard_set: shard_tables_cache, meta_client, topology: Default::default(), }) @@ -235,20 +242,21 @@ impl Inner { Ok(resp) } - async fn open_shard(&self, shard_info: &ShardInfo) -> Result { - if let Some(tables_of_shard) = self.shard_tables_cache.get(shard_info.id) { - if tables_of_shard.shard_info.version == shard_info.version { + async fn open_shard(&self, shard_info: &ShardInfo) -> Result { + if let Some(shard) = self.shard_set.get(shard_info.id) { + let inner = shard.read().await; + if inner.shard_info.version == shard_info.version { info!( "No need to open the exactly same shard again, shard_info:{:?}", shard_info ); - return Ok(tables_of_shard); + return Ok(shard.clone()); } ensure!( - tables_of_shard.shard_info.version < shard_info.version, + inner.shard_info.version < shard_info.version, OpenShard { shard_id: shard_info.id, - msg: format!("open a shard with a smaller version, curr_shard_info:{:?}, new_shard_info:{:?}", tables_of_shard.shard_info, shard_info), + msg: format!("open a shard with a smaller version, curr_shard_info:{:?}, new_shard_info:{:?}", inner.shard_info, shard_info), } ); } @@ -282,95 +290,20 @@ impl Inner { msg: "shard tables are missing from the response", })?; - self.shard_tables_cache.insert(tables_of_shard.clone()); + let shard_id = tables_of_shard.shard_info.id; + let shard = Arc::new(tokio::sync::RwLock::new(tables_of_shard.into())); + self.shard_set.insert(shard_id, shard.clone()); - Ok(tables_of_shard) + Ok(shard) } - fn close_shard(&self, shard_id: ShardId) -> Result { - self.shard_tables_cache + fn close_shard(&self, shard_id: ShardId) -> Result { + self.shard_set .remove(shard_id) .with_context(|| ShardNotFound { msg: format!("close non-existent shard, shard_id:{shard_id}"), }) } - - #[inline] - fn freeze_shard(&self, shard_id: ShardId) -> Result { - self.shard_tables_cache - .freeze(shard_id) - .with_context(|| ShardNotFound { - msg: format!("try to freeze a non-existent shard, shard_id:{shard_id}"), - }) - } - - fn create_table_on_shard(&self, req: &CreateTableOnShardRequest) -> Result<()> { - self.insert_table_to_shard(req.update_shard_info.clone(), req.table_info.clone()) - } - - fn drop_table_on_shard(&self, req: &DropTableOnShardRequest) -> Result<()> { - self.remove_table_from_shard(req.update_shard_info.clone(), req.table_info.clone()) - } - - fn open_table_on_shard(&self, req: &OpenTableOnShardRequest) -> Result<()> { - self.insert_table_to_shard(req.update_shard_info.clone(), req.table_info.clone()) - } - - fn close_table_on_shard(&self, req: &CloseTableOnShardRequest) -> Result<()> { - self.remove_table_from_shard(req.update_shard_info.clone(), req.table_info.clone()) - } - - fn insert_table_to_shard( - &self, - update_shard_info: Option, - table_info: Option, - ) -> Result<()> { - let update_shard_info = update_shard_info.context(ShardNotFound { - msg: "update shard info is missing", - })?; - let curr_shard_info = update_shard_info.curr_shard_info.context(ShardNotFound { - msg: "current shard info is missing", - })?; - let table_info = table_info.context(TableNotFound { - msg: "table info is missing", - })?; - - self.shard_tables_cache.try_insert_table_to_shard( - update_shard_info.prev_version, - ShardInfo::from(&curr_shard_info), - TableInfo::try_from(table_info) - .box_err() - .context(Internal { - msg: "Failed to parse tableInfo", - })?, - ) - } - - fn remove_table_from_shard( - &self, - update_shard_info: Option, - table_info: Option, - ) -> Result<()> { - let update_shard_info = update_shard_info.context(ShardNotFound { - msg: "update shard info is missing", - })?; - let curr_shard_info = update_shard_info.curr_shard_info.context(ShardNotFound { - msg: "current shard info is missing", - })?; - let table_info = table_info.context(TableNotFound { - msg: "table info is missing", - })?; - - self.shard_tables_cache.try_remove_table_from_shard( - update_shard_info.prev_version, - ShardInfo::from(&curr_shard_info), - TableInfo::try_from(table_info) - .box_err() - .context(Internal { - msg: "Failed to parse tableInfo", - })?, - ) - } } #[async_trait] @@ -406,34 +339,14 @@ impl Cluster for ClusterImpl { Ok(()) } - async fn open_shard(&self, shard_info: &ShardInfo) -> Result { + async fn open_shard(&self, shard_info: &ShardInfo) -> Result { self.inner.open_shard(shard_info).await } - async fn close_shard(&self, shard_id: ShardId) -> Result { + async fn close_shard(&self, shard_id: ShardId) -> Result { self.inner.close_shard(shard_id) } - async fn freeze_shard(&self, shard_id: ShardId) -> Result { - self.inner.freeze_shard(shard_id) - } - - async fn create_table_on_shard(&self, req: &CreateTableOnShardRequest) -> Result<()> { - self.inner.create_table_on_shard(req) - } - - async fn drop_table_on_shard(&self, req: &DropTableOnShardRequest) -> Result<()> { - self.inner.drop_table_on_shard(req) - } - - async fn open_table_on_shard(&self, req: &OpenTableOnShardRequest) -> Result<()> { - self.inner.open_table_on_shard(req) - } - - async fn close_table_on_shard(&self, req: &CloseTableOnShardRequest) -> Result<()> { - self.inner.close_table_on_shard(req) - } - async fn route_tables(&self, req: &RouteTablesRequest) -> Result { self.inner.route_tables(req).await } diff --git a/cluster/src/lib.rs b/cluster/src/lib.rs index e91e4cc862..872500e202 100644 --- a/cluster/src/lib.rs +++ b/cluster/src/lib.rs @@ -26,10 +26,12 @@ use meta_client::types::{ use shard_lock_manager::ShardLockManagerRef; use snafu::{Backtrace, Snafu}; +use crate::shard_set::ShardRef; + pub mod cluster_impl; pub mod config; pub mod shard_lock_manager; -pub mod shard_tables_cache; +pub mod shard_set; #[allow(dead_code)] pub mod topology; @@ -132,19 +134,14 @@ pub struct ClusterNodesResp { pub trait Cluster { async fn start(&self) -> Result<()>; async fn stop(&self) -> Result<()>; - async fn open_shard(&self, shard_info: &ShardInfo) -> Result; + async fn open_shard(&self, shard_info: &ShardInfo) -> Result; /// Close the shard. /// /// Return error if the shard is not found. - async fn close_shard(&self, req: ShardId) -> Result; + async fn close_shard(&self, req: ShardId) -> Result; /// Freeze the shard to reject create/drop table on the shard. /// /// Return error if the shard is not found. - async fn freeze_shard(&self, req: ShardId) -> Result; - async fn create_table_on_shard(&self, req: &CreateTableOnShardRequest) -> Result<()>; - async fn drop_table_on_shard(&self, req: &DropTableOnShardRequest) -> Result<()>; - async fn open_table_on_shard(&self, req: &OpenTableOnShardRequest) -> Result<()>; - async fn close_table_on_shard(&self, req: &CloseTableOnShardRequest) -> Result<()>; async fn route_tables(&self, req: &RouteTablesRequest) -> Result; async fn fetch_nodes(&self) -> Result; fn shard_lock_manager(&self) -> ShardLockManagerRef; diff --git a/cluster/src/shard_set.rs b/cluster/src/shard_set.rs new file mode 100644 index 0000000000..0a6e3fcbb2 --- /dev/null +++ b/cluster/src/shard_set.rs @@ -0,0 +1,176 @@ +// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. + +use std::{ + collections::HashMap, + sync::{Arc, RwLock}, +}; + +use meta_client::types::{ShardId, ShardInfo, ShardVersion, TableInfo, TablesOfShard}; +use snafu::{ensure, OptionExt}; + +use crate::{ + Result, ShardNotFound, ShardVersionMismatch, TableAlreadyExists, TableNotFound, + UpdateFrozenShard, +}; + +/// [ShardTablesCache] caches the information about tables and shards, and the +/// relationship between them is: one shard -> multiple tables. +#[derive(Debug, Default, Clone)] +pub struct ShardSet { + inner: Arc>, +} + +#[derive(Debug, Clone)] +pub struct TableWithShards { + pub table_info: TableInfo, + pub shard_infos: Vec, +} + +impl ShardSet { + // Fetch all the shard infos. + pub fn all_shards(&self) -> Vec { + self.inner.read().unwrap().all_shard_infos() + } + + // Get the shard by its id. + pub fn get(&self, shard_id: ShardId) -> Option { + self.inner.read().unwrap().get(shard_id) + } + + /// Remove the shard. + pub fn remove(&self, shard_id: ShardId) -> Option { + self.inner.write().unwrap().remove(shard_id) + } + + /// Insert the tables of one shard. + pub fn insert(&self, shard_id: ShardId, shard: ShardRef) { + self.inner.write().unwrap().insert(shard_id, shard) + } +} + +#[derive(Debug)] +pub struct Shard { + /// Shard info + pub shard_info: ShardInfo, + + /// Tables in shard + pub tables: Vec, + + /// Flag indicating that further updates are prohibited + pub frozen: bool, +} + +impl Shard { + fn freeze(&mut self) { + self.frozen = true; + } + + fn try_insert_table( + &mut self, + prev_shard_version: ShardVersion, + curr_shard: ShardInfo, + new_table: TableInfo, + ) -> Result<()> { + ensure!( + !self.frozen, + UpdateFrozenShard { + shard_id: curr_shard.id, + } + ); + + ensure!( + self.shard_info.version == prev_shard_version, + ShardVersionMismatch { + shard_info: self.shard_info.clone(), + expect_version: prev_shard_version, + } + ); + + let table = self.tables.iter().find(|v| v.id == new_table.id); + ensure!( + table.is_none(), + TableAlreadyExists { + msg: "the table to insert has already existed", + } + ); + + // Update tables of shard. + self.shard_info = curr_shard; + self.tables.push(new_table); + + Ok(()) + } + + fn try_remove_table( + &mut self, + prev_shard_version: ShardVersion, + curr_shard: ShardInfo, + new_table: TableInfo, + ) -> Result<()> { + ensure!( + !self.frozen, + UpdateFrozenShard { + shard_id: curr_shard.id, + } + ); + + ensure!( + self.shard_info.version == prev_shard_version, + ShardVersionMismatch { + shard_info: self.shard_info.clone(), + expect_version: prev_shard_version, + } + ); + + let table_idx = self + .tables + .iter() + .position(|v| v.id == new_table.id) + .with_context(|| TableNotFound { + msg: format!("the table to remove is not found, table:{new_table:?}"), + })?; + + // Update tables of shard. + self.shard_info = curr_shard; + self.tables.swap_remove(table_idx); + + Ok(()) + } +} + +pub type ShardRef = Arc>; + +impl From for Shard { + fn from(value: TablesOfShard) -> Self { + Shard { + shard_info: value.shard_info, + tables: value.tables, + frozen: false, + } + } +} + +#[derive(Debug, Default)] +struct Inner { + // Tables organized by shard. + // TODO: The shard roles should be also taken into considerations. + shards: HashMap, +} + +impl Inner { + fn all_shard_infos(&self) -> Vec { + self.shards.values().cloned().collect() + } + + fn get(&self, shard_id: ShardId) -> Option { + self.shards.get(&shard_id).cloned() + } + + fn remove(&mut self, shard_id: ShardId) -> Option { + self.shards.remove(&shard_id) + } + + fn insert(&mut self, shard_id: ShardId, shard: ShardRef) { + self.shards.insert(shard_id, shard); + } +} diff --git a/cluster/src/shard_tables_cache.rs b/cluster/src/shard_tables_cache.rs deleted file mode 100644 index e38f7c1f92..0000000000 --- a/cluster/src/shard_tables_cache.rs +++ /dev/null @@ -1,279 +0,0 @@ -// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. - -use std::{ - collections::HashMap, - sync::{Arc, RwLock}, -}; - -use meta_client::types::{ShardId, ShardInfo, ShardVersion, TableInfo, TablesOfShard}; -use snafu::{ensure, OptionExt}; - -use crate::{ - Result, ShardNotFound, ShardVersionMismatch, TableAlreadyExists, TableNotFound, - UpdateFrozenShard, -}; - -/// [ShardTablesCache] caches the information about tables and shards, and the -/// relationship between them is: one shard -> multiple tables. -#[derive(Debug, Default, Clone)] -pub struct ShardTablesCache { - inner: Arc>, -} - -#[derive(Debug, Clone)] -pub struct TableWithShards { - pub table_info: TableInfo, - pub shard_infos: Vec, -} - -impl ShardTablesCache { - pub fn find_table_by_name( - &self, - catalog: &str, - schema: &str, - table: &str, - ) -> Option { - self.inner - .read() - .unwrap() - .find_table_by_name(catalog, schema, table) - } - - // Fetch all the shard infos. - pub fn all_shard_infos(&self) -> Vec { - self.inner.read().unwrap().all_shard_infos() - } - - // Get the shard by its id. - pub fn get(&self, shard_id: ShardId) -> Option { - self.inner.read().unwrap().get(shard_id) - } - - /// Remove the shard. - pub fn remove(&self, shard_id: ShardId) -> Option { - self.inner.write().unwrap().remove(shard_id) - } - - /// Insert the tables of one shard. - pub fn insert(&self, tables_of_shard: TablesOfShard) { - self.inner.write().unwrap().insert(tables_of_shard) - } - - /// Freeze the shard. - pub fn freeze(&self, shard_id: ShardId) -> Option { - self.inner.write().unwrap().freeze(shard_id) - } - - /// Try to insert a new table to the shard with a newer version. - /// - /// It will fail if: - /// - the shard doesn't exist, or - /// - the shard version in the cache is not equal to the provided - /// `prev_shard_version`, or - /// - the table already exists. - pub fn try_insert_table_to_shard( - &self, - prev_shard_version: ShardVersion, - curr_shard: ShardInfo, - new_table: TableInfo, - ) -> Result<()> { - self.inner.write().unwrap().try_insert_table_to_shard( - prev_shard_version, - curr_shard, - new_table, - ) - } - - /// Try to remove a table from the shard with a newer version. - /// - /// It will fail if: - /// - the shard doesn't exist, or - /// - the shard version in the cache is not equal to the provided - /// `prev_shard_version`, or - /// - the table doesn't exist. - pub fn try_remove_table_from_shard( - &self, - prev_shard_version: ShardVersion, - curr_shard: ShardInfo, - new_table: TableInfo, - ) -> Result<()> { - self.inner.write().unwrap().try_remove_table_from_shard( - prev_shard_version, - curr_shard, - new_table, - ) - } -} - -#[derive(Clone, Debug)] -struct TablesOfShardCacheEntry { - entry: TablesOfShard, - frozen: bool, -} - -#[derive(Debug, Default)] -struct Inner { - // Tables organized by shard. - // TODO: The shard roles should be also taken into considerations. - tables_by_shard: HashMap, -} - -impl Inner { - pub fn find_table_by_name( - &self, - _catalog_name: &str, - schema_name: &str, - table_name: &str, - ) -> Option { - // TODO: We need a more efficient way to find TableWithShards. - let mut table_info = None; - let mut shard_infos = vec![]; - for tables_of_shard in self.tables_by_shard.values() { - if let Some(v) = tables_of_shard - .entry - .tables - .iter() - .find(|table| table.schema_name == schema_name && table.name == table_name) - { - if table_info.is_none() { - table_info = Some(v.clone()); - } - - shard_infos.push(tables_of_shard.entry.shard_info.clone()); - } - } - - table_info.map(|v| TableWithShards { - table_info: v, - shard_infos, - }) - } - - fn all_shard_infos(&self) -> Vec { - self.tables_by_shard - .values() - .map(|v| v.entry.shard_info.clone()) - .collect() - } - - fn get(&self, shard_id: ShardId) -> Option { - self.tables_by_shard.get(&shard_id).map(|v| v.entry.clone()) - } - - fn remove(&mut self, shard_id: ShardId) -> Option { - self.tables_by_shard.remove(&shard_id).map(|v| v.entry) - } - - fn freeze(&mut self, shard_id: ShardId) -> Option { - let tables_of_shard = self.tables_by_shard.get_mut(&shard_id)?; - tables_of_shard.frozen = true; - - Some(tables_of_shard.entry.clone()) - } - - fn insert(&mut self, tables_of_shard: TablesOfShard) { - let shard_id = tables_of_shard.shard_info.id; - let entry = TablesOfShardCacheEntry { - entry: tables_of_shard, - frozen: false, - }; - self.tables_by_shard.insert(shard_id, entry); - } - - fn try_insert_table_to_shard( - &mut self, - prev_shard_version: ShardVersion, - curr_shard: ShardInfo, - new_table: TableInfo, - ) -> Result<()> { - let tables_of_shard = self - .tables_by_shard - .get_mut(&curr_shard.id) - .with_context(|| ShardNotFound { - msg: format!( - "insert table to a non-existent shard, shard_id:{}", - curr_shard.id - ), - })?; - - ensure!( - !tables_of_shard.frozen, - UpdateFrozenShard { - shard_id: curr_shard.id, - } - ); - - ensure!( - tables_of_shard.entry.shard_info.version == prev_shard_version, - ShardVersionMismatch { - shard_info: tables_of_shard.entry.shard_info.clone(), - expect_version: prev_shard_version, - } - ); - - let table = tables_of_shard - .entry - .tables - .iter() - .find(|v| v.id == new_table.id); - ensure!( - table.is_none(), - TableAlreadyExists { - msg: "the table to insert has already existed", - } - ); - - // Update tables of shard. - tables_of_shard.entry.shard_info = curr_shard; - tables_of_shard.entry.tables.push(new_table); - - Ok(()) - } - - fn try_remove_table_from_shard( - &mut self, - prev_shard_version: ShardVersion, - curr_shard: ShardInfo, - new_table: TableInfo, - ) -> Result<()> { - let tables_of_shard = self - .tables_by_shard - .get_mut(&curr_shard.id) - .with_context(|| ShardNotFound { - msg: format!( - "remove table from a non-existent shard, shard_id:{}", - curr_shard.id - ), - })?; - - ensure!( - !tables_of_shard.frozen, - UpdateFrozenShard { - shard_id: curr_shard.id, - } - ); - - ensure!( - tables_of_shard.entry.shard_info.version == prev_shard_version, - ShardVersionMismatch { - shard_info: tables_of_shard.entry.shard_info.clone(), - expect_version: prev_shard_version, - } - ); - - let table_idx = tables_of_shard - .entry - .tables - .iter() - .position(|v| v.id == new_table.id) - .with_context(|| TableNotFound { - msg: format!("the table to remove is not found, table:{new_table:?}"), - })?; - - // Update tables of shard. - tables_of_shard.entry.shard_info = curr_shard; - tables_of_shard.entry.tables.swap_remove(table_idx); - - Ok(()) - } -} diff --git a/src/setup.rs b/src/setup.rs index 286649cea5..6d950aa8fd 100644 --- a/src/setup.rs +++ b/src/setup.rs @@ -11,9 +11,7 @@ use analytic_engine::{ }; use catalog::{manager::ManagerRef, schema::OpenOptions, table_operator::TableOperator}; use catalog_impls::{table_based::TableBasedManager, volatile, CatalogManagerImpl}; -use cluster::{ - cluster_impl::ClusterImpl, config::ClusterConfig, shard_tables_cache::ShardTablesCache, -}; +use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_tables_cache::ShardSet}; use common_util::runtime; use df_operator::registry::FunctionRegistryImpl; use interpreters::table_manipulator::{catalog_based, meta_based}; @@ -211,7 +209,7 @@ async fn build_with_meta( .await .expect("fail to build meta client"); - let shard_tables_cache = ShardTablesCache::default(); + let shard_tables_cache = ShardSet::default(); let cluster = { let cluster_impl = ClusterImpl::try_new( endpoint, From ff576e10ab938d8035c8e8b2cdf2c654b1c9577a Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 7 Jul 2023 13:10:56 +0800 Subject: [PATCH 02/15] modify related logic, and remove useless `find_table_by_name` from `ShardSet`. --- catalog_impls/src/volatile.rs | 29 ++++++++++++++++++----------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/catalog_impls/src/volatile.rs b/catalog_impls/src/volatile.rs index 236418c022..4ea156c04f 100644 --- a/catalog_impls/src/volatile.rs +++ b/catalog_impls/src/volatile.rs @@ -188,7 +188,7 @@ struct SchemaImpl { /// Schema name schema_name: String, schema_id: SchemaId, - shard_tables_cache: ShardSet, + shard_set: ShardSet, /// Tables of schema tables: RwLock>, /// Guard for creating/dropping table @@ -206,7 +206,7 @@ impl SchemaImpl { catalog_name, schema_name, schema_id, - shard_tables_cache, + shard_set: shard_tables_cache, tables: Default::default(), create_table_mutex: Mutex::new(()), } @@ -303,17 +303,24 @@ impl Schema for SchemaImpl { // Do real create table. // Partition table is not stored in ShardTableManager. if request.partition_info.is_none() { - let _ = self - .shard_tables_cache - .find_table_by_name( - &request.catalog_name, - &request.schema_name, - &request.table_name, - ) + let shard = + self.shard_set + .get(request.shard_id) + .with_context(|| schema::CreateTable { + request: request.clone(), + msg: format!("shard not found"), + })?; + + let shard = shard.read().await; + let _ = shard + .tables + .iter() + .find(|table| { + table.schema_name == request.schema_name && table.name == request.table_name + }) .with_context(|| schema::CreateTable { request: request.clone(), - msg: format!("table with shards is not found in the ShardTableManager, catalog_name:{}, schema_name:{}, table_name:{}", - request.catalog_name, request.schema_name, request.table_name), + msg: format!("table not found in shard"), })?; } let request = request.into_engine_create_request(None, self.schema_id); From 95246aedafdd4492a31bb480ca9fcfddde75ed85 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 7 Jul 2023 13:22:25 +0800 Subject: [PATCH 03/15] add `get_shard` method into `Cluster`. --- cluster/src/cluster_impl.rs | 8 ++++++++ cluster/src/lib.rs | 15 +++++++++++---- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/cluster/src/cluster_impl.rs b/cluster/src/cluster_impl.rs index a9bdbb7df1..090ca18e1b 100644 --- a/cluster/src/cluster_impl.rs +++ b/cluster/src/cluster_impl.rs @@ -297,6 +297,10 @@ impl Inner { Ok(shard) } + fn get_shard(&self, shard_id: ShardId) -> Option { + self.shard_set.get(shard_id) + } + fn close_shard(&self, shard_id: ShardId) -> Result { self.shard_set .remove(shard_id) @@ -343,6 +347,10 @@ impl Cluster for ClusterImpl { self.inner.open_shard(shard_info).await } + fn get_shard(&self, shard_id: ShardId) -> Option { + self.inner.get_shard(shard_id) + } + async fn close_shard(&self, shard_id: ShardId) -> Result { self.inner.close_shard(shard_id) } diff --git a/cluster/src/lib.rs b/cluster/src/lib.rs index 872500e202..f856e39c71 100644 --- a/cluster/src/lib.rs +++ b/cluster/src/lib.rs @@ -134,14 +134,21 @@ pub struct ClusterNodesResp { pub trait Cluster { async fn start(&self) -> Result<()>; async fn stop(&self) -> Result<()>; + + /// Fetch related information and open shard. async fn open_shard(&self, shard_info: &ShardInfo) -> Result; - /// Close the shard. + + /// Get shard. /// - /// Return error if the shard is not found. - async fn close_shard(&self, req: ShardId) -> Result; - /// Freeze the shard to reject create/drop table on the shard. + /// If target shard has opened in cluster, return it. Otherwise, return + /// None. + fn get_shard(&self, shard_id: ShardId) -> Option; + + /// Close shard. /// /// Return error if the shard is not found. + async fn close_shard(&self, shard_id: ShardId) -> Result; + async fn route_tables(&self, req: &RouteTablesRequest) -> Result; async fn fetch_nodes(&self) -> Result; fn shard_lock_manager(&self) -> ShardLockManagerRef; From d3f0d363d1aedf87dc821c17a3fdaebd94950372 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 7 Jul 2023 15:40:00 +0800 Subject: [PATCH 04/15] tmp. --- ...fix-bug-about-empty-region-s-opening.patch | 99 +++ kafka-delete.patch | 622 ++++++++++++++++++ router/src/cluster_based.rs | 32 +- server/src/grpc/meta_event_service/mod.rs | 55 +- 4 files changed, 764 insertions(+), 44 deletions(-) create mode 100644 0001-fix-bug-about-empty-region-s-opening.patch create mode 100644 kafka-delete.patch diff --git a/0001-fix-bug-about-empty-region-s-opening.patch b/0001-fix-bug-about-empty-region-s-opening.patch new file mode 100644 index 0000000000..2712bc9660 --- /dev/null +++ b/0001-fix-bug-about-empty-region-s-opening.patch @@ -0,0 +1,99 @@ +From bfcc7c79f6d3a7a92ec720358742880fd9a54147 Mon Sep 17 00:00:00 2001 +From: kamille +Date: Fri, 30 Jun 2023 02:23:45 +0800 +Subject: [PATCH] fix bug about empty region's opening. + +--- + wal/src/message_queue_impl/region.rs | 48 +++++++++++++++++++++++++--- + 1 file changed, 43 insertions(+), 5 deletions(-) + +diff --git a/wal/src/message_queue_impl/region.rs b/wal/src/message_queue_impl/region.rs +index 321c1978..d1995387 100644 +--- a/wal/src/message_queue_impl/region.rs ++++ b/wal/src/message_queue_impl/region.rs +@@ -224,7 +224,17 @@ impl Region { + namespace, region_id + ); + +- // Fetch high watermark and check. ++ // Fetch earliest, high watermark and check. ++ let earliest = message_queue ++ .fetch_offset(meta_topic, OffsetType::EarliestOffset) ++ .await ++ .box_err() ++ .context(OpenWithCause { ++ namespace, ++ region_id, ++ msg: "failed while recover from meta", ++ })?; ++ + let high_watermark = message_queue + .fetch_offset(meta_topic, OffsetType::HighWaterMark) + .await +@@ -235,9 +245,19 @@ impl Region { + msg: "failed while recover from meta", + })?; + +- if high_watermark == 0 { +- debug!("Meta topic is empty, it just needs to recover from log topic, namespace:{}, region id:{}", namespace, region_id); +- return Ok(None); ++ if earliest == high_watermark { ++ if high_watermark == 0 { ++ info!("Recover region meta from meta, found empty meta topic, just need to recover from log topic, namespace:{}, region id:{}", ++ namespace, region_id); ++ return Ok(None); ++ } ++ ++ return OpenNoCause { ++ namespace, ++ region_id, ++ msg: "region meta impossible to be empty when having written logs", ++ } ++ .fail(); + } + + // Fetch snapshot from meta topic(just fetch the last snapshot). +@@ -313,6 +333,9 @@ impl Region { + }); + + let region_safe_delete_offset = if min_safe_delete_offset == i64::MAX { ++ info!("Recover region meta from meta, min_safe_delete_offset not exist, region_meta_snapshot:{:?}, namespace:{}, region id:{}", ++ value, namespace, region_id); ++ + None + } else { + Some(min_safe_delete_offset) +@@ -353,6 +376,17 @@ impl Region { + // FIXME: should not judge whether topic is empty or not by caller. + // The consumer iterator should return immediately rather than hanging when + // topic empty. ++ // Fetch earliest, high watermark and check. ++ let earliest = message_queue ++ .fetch_offset(log_topic, OffsetType::EarliestOffset) ++ .await ++ .box_err() ++ .context(OpenWithCause { ++ namespace, ++ region_id, ++ msg: "failed while recover from log", ++ })?; ++ + let high_watermark = message_queue + .fetch_offset(log_topic, OffsetType::HighWaterMark) + .await +@@ -362,7 +396,11 @@ impl Region { + region_id, + msg: "failed while recover from log", + })?; +- if high_watermark == 0 { ++ ++ if earliest == high_watermark { ++ info!("Recover region meta from log, found empty log topic, namespace:{}, region_id:{}, earliest:{}, high_watermark:{}", ++ namespace, region_id, earliest, high_watermark ++ ); + return Ok(()); + } + +-- +2.32.1 (Apple Git-133) + diff --git a/kafka-delete.patch b/kafka-delete.patch new file mode 100644 index 0000000000..97a587993c --- /dev/null +++ b/kafka-delete.patch @@ -0,0 +1,622 @@ +From dee7d12ee23245efdd08efd4fd8f9b18ddc22e4b Mon Sep 17 00:00:00 2001 +From: kamille +Date: Tue, 27 Jun 2023 14:27:53 +0800 +Subject: [PATCH 1/4] fix deleted sequence updating and recover begin offset + computing. + +--- + wal/src/message_queue_impl/region.rs | 71 +++++++++++--------- + wal/src/message_queue_impl/region_context.rs | 19 +++--- + 2 files changed, 50 insertions(+), 40 deletions(-) + +diff --git a/wal/src/message_queue_impl/region.rs b/wal/src/message_queue_impl/region.rs +index 37beea03..5ac07340 100644 +--- a/wal/src/message_queue_impl/region.rs ++++ b/wal/src/message_queue_impl/region.rs +@@ -148,6 +148,7 @@ impl Region { + region_id, + msg: "failed while trying to create topic", + })?; ++ + message_queue + .create_topic_if_not_exist(&meta_topic) + .await +@@ -160,7 +161,7 @@ impl Region { + + // Build region meta. + let mut region_meta_builder = RegionContextBuilder::new(region_id); +- let high_watermark_in_snapshot = Self::recover_region_meta_from_meta( ++ let region_safe_delete_offset = Self::recover_region_meta_from_meta( + namespace, + region_id, + message_queue.as_ref(), +@@ -174,7 +175,7 @@ impl Region { + namespace, + region_id, + message_queue.as_ref(), +- high_watermark_in_snapshot, ++ region_safe_delete_offset, + &log_topic, + &log_encoding, + &mut region_meta_builder, +@@ -217,7 +218,7 @@ impl Region { + meta_topic: &str, + meta_encoding: &MetaEncoding, + builder: &mut RegionContextBuilder, +- ) -> Result { ++ ) -> Result> { + info!( + "Recover region meta from meta, namespace:{}, region id:{}", + namespace, region_id +@@ -236,7 +237,7 @@ impl Region { + + if high_watermark == 0 { + debug!("Meta topic is empty, it just needs to recover from log topic, namespace:{}, region id:{}", namespace, region_id); +- return Ok(0); ++ return Ok(None); + } + + // Fetch snapshot from meta topic(just fetch the last snapshot). +@@ -304,10 +305,18 @@ impl Region { + msg: "failed while recover from meta", + })?; + +- let high_watermark_in_snapshot = value +- .entries +- .iter() +- .fold(0, |hw, entry| cmp::max(hw, entry.current_high_watermark)); ++ let min_safe_delete_offset = value.entries.iter().fold(i64::MAX, |min_offset, entry| { ++ match entry.safe_delete_offset { ++ Some(offset) => cmp::min(min_offset, offset), ++ None => min_offset, ++ } ++ }); ++ ++ let region_safe_delete_offset = if min_safe_delete_offset == i64::MAX { ++ None ++ } else { ++ Some(min_safe_delete_offset) ++ }; + + builder + .apply_region_meta_snapshot(value) +@@ -318,24 +327,32 @@ impl Region { + msg: "failed while recover from meta", + })?; + +- Ok(high_watermark_in_snapshot) ++ Ok(region_safe_delete_offset) + } + + async fn recover_region_meta_from_log( + namespace: &str, + region_id: u64, + message_queue: &M, +- start_offset: Offset, ++ region_safe_delete_offset: Option, + log_topic: &str, + log_encoding: &CommonLogEncoding, + builder: &mut RegionContextBuilder, + ) -> Result<()> { + info!( +- "Recover region meta from log, namespace:{}, region id:{}, start offset:{}", +- namespace, region_id, start_offset ++ "Recover region meta from log, namespace:{}, region_id:{}, region_safe_delete_offset:{:?}", ++ namespace, region_id, region_safe_delete_offset + ); + +- // Fetch high watermark and check. ++ let start_offset = match region_safe_delete_offset { ++ Some(offset) => StartOffset::At(offset), ++ None => StartOffset::Earliest, ++ }; ++ ++ // Fetch snapshot from meta topic(just fetch the last snapshot). ++ // FIXME: should not judge whether topic is empty or not by caller. ++ // The consumer iterator should return immediately rather than hanging when ++ // topic empty. + let high_watermark = message_queue + .fetch_offset(log_topic, OffsetType::HighWaterMark) + .await +@@ -345,19 +362,12 @@ impl Region { + region_id, + msg: "failed while recover from log", + })?; +- +- ensure!(start_offset <= high_watermark, OpenNoCause { namespace , region_id, msg: format!( +- "failed while recover from log, start offset should be less than or equal to high watermark, now are:{start_offset} and {high_watermark}") +- }); +- +- if start_offset == high_watermark { +- debug!("No region meta delta from log topic is needed, just return, namespace:{}, region id:{}", namespace, region_id); ++ if high_watermark == 0 { + return Ok(()); + } + +- // Fetch snapshot from meta topic(just fetch the last snapshot). + let mut iter = message_queue +- .consume(log_topic, StartOffset::At(start_offset)) ++ .consume(log_topic, start_offset) + .await + .box_err() + .context(OpenWithCause { +@@ -637,7 +647,7 @@ impl Region { + } + + /// Return snapshot, just used for test. +- #[allow(unused)] ++ #[allow(dead_code)] + async fn make_meta_snapshot(&self) -> RegionMetaSnapshot { + let inner = self.inner.write().await; + inner.make_meta_snapshot().await +@@ -907,7 +917,6 @@ mod tests { + }; + + #[tokio::test] +- #[ignore] + async fn test_region_kafka_impl() { + // Test region + let mut config = Config::default(); +@@ -962,7 +971,7 @@ mod tests { + .write(&WriteContext::default(), test_log_batch) + .await + .unwrap(); +- assert_eq!(sequence_num, test_log_batch.len() as u64 - 1); ++ assert_eq!(sequence_num, test_log_batch.len() as u64); + + mixed_test_payloads.extend_from_slice(test_payloads); + } +@@ -1079,18 +1088,18 @@ mod tests { + .write(&WriteContext::default(), test_log_batch) + .await + .unwrap(); +- assert_eq!(sequence_num, test_log_batch.len() as u64 - 1); ++ assert_eq!(sequence_num, test_log_batch.len() as u64); + let sequence_num = test_context + .region + .write(&WriteContext::default(), test_log_batch) + .await + .unwrap(); +- assert_eq!(sequence_num, test_log_batch.len() as u64 * 2 - 1); ++ assert_eq!(sequence_num, test_log_batch.len() as u64 * 2); + + // Mark deleted. + test_context + .region +- .mark_delete_to(table_id, test_log_batch.len() as u64) ++ .mark_delete_to(table_id, test_log_batch.len() as u64 + 1) + .await + .unwrap(); + let table_meta = test_context +@@ -1101,11 +1110,11 @@ mod tests { + .unwrap(); + assert_eq!( + table_meta.next_sequence_num, +- test_log_batch.len() as u64 * 2 ++ test_log_batch.len() as u64 * 2 + 1, + ); + assert_eq!( + table_meta.latest_marked_deleted, +- test_log_batch.len() as u64 ++ test_log_batch.len() as u64 + 1 + ); + assert_eq!( + table_meta.current_high_watermark, +@@ -1151,7 +1160,7 @@ mod tests { + .write(&WriteContext::default(), test_log_batch) + .await + .unwrap(); +- assert_eq!(sequence_num, test_log_batch.len() as u64 - 1); ++ assert_eq!(sequence_num, test_log_batch.len() as u64); + + mixed_test_payloads.extend_from_slice(test_payloads); + } +diff --git a/wal/src/message_queue_impl/region_context.rs b/wal/src/message_queue_impl/region_context.rs +index 3b611e59..441eb93c 100644 +--- a/wal/src/message_queue_impl/region_context.rs ++++ b/wal/src/message_queue_impl/region_context.rs +@@ -1,4 +1,4 @@ +-// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. ++// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. + + //! Region context + +@@ -310,7 +310,7 @@ impl TableMeta { + // The `start_sequence_offset_mapping` is possible to be incomplete during + // recovery. + let offset = inner.start_sequence_offset_mapping.get(&sequence_num); +- if offset.is_none() && inner.next_sequence_num != inner.latest_marked_deleted { ++ if offset.is_none() && inner.next_sequence_num != sequence_num { + warn!("Start sequence offset mapping is incomplete, + just not update the marked deleted sequence in this flush, new marked deleted, sequence num:{}, previous:{}", + sequence_num, inner.latest_marked_deleted); +@@ -469,14 +469,12 @@ impl OffsetRange { + } + + /// Builder for `RegionMeta` +-#[allow(unused)] + #[derive(Debug)] + pub struct RegionContextBuilder { + region_id: u64, + table_metas: HashMap, + } + +-#[allow(unused)] + impl RegionContextBuilder { + pub fn new(region_id: u64) -> Self { + Self { +@@ -503,6 +501,7 @@ impl RegionContextBuilder { + pub fn apply_region_meta_delta(&mut self, delta: RegionMetaDelta) -> Result<()> { + debug!("Apply region meta delta, delta:{:?}", delta); + ++ // It is likely that snapshot not exist(e.g. no table has ever flushed). + let mut table_meta = self + .table_metas + .entry(delta.table_id) +@@ -522,9 +521,13 @@ impl RegionContextBuilder { + ) }); + table_meta.current_high_watermark = delta.offset + 1; + +- table_meta +- .start_sequence_offset_mapping +- .insert(delta.sequence_num, delta.offset); ++ // Because recover from the `region_safe_delete_offset`, some outdated logs will ++ // be loaded. ++ if delta.sequence_num >= table_meta.latest_marked_deleted { ++ table_meta ++ .start_sequence_offset_mapping ++ .insert(delta.sequence_num, delta.offset); ++ } + + Ok(()) + } +@@ -557,7 +560,6 @@ impl RegionContextBuilder { + } + } + +-#[allow(unused)] + #[derive(Debug, Clone)] + pub struct RegionMetaDelta { + table_id: TableId, +@@ -565,7 +567,6 @@ pub struct RegionMetaDelta { + offset: Offset, + } + +-#[allow(unused)] + impl RegionMetaDelta { + pub fn new(table_id: TableId, sequence_num: SequenceNumber, offset: Offset) -> Self { + Self { +-- +2.32.1 (Apple Git-133) + + +From 17fefa5de7ad20a3b262285f7432a3430d75520b Mon Sep 17 00:00:00 2001 +From: kamille +Date: Tue, 27 Jun 2023 14:39:56 +0800 +Subject: [PATCH 2/4] fix kafka related tests and clippy. + +--- + .../message_queue/src/kafka/kafka_impl.rs | 1 + + components/message_queue/src/tests/cases.rs | 3 +-- + wal/src/kv_encoder.rs | 19 +++---------------- + wal/src/message_queue_impl/encoding.rs | 14 ++++++-------- + wal/src/message_queue_impl/region.rs | 1 + + 5 files changed, 12 insertions(+), 26 deletions(-) + +diff --git a/components/message_queue/src/kafka/kafka_impl.rs b/components/message_queue/src/kafka/kafka_impl.rs +index 0d69d5e3..0fdfb462 100644 +--- a/components/message_queue/src/kafka/kafka_impl.rs ++++ b/components/message_queue/src/kafka/kafka_impl.rs +@@ -284,6 +284,7 @@ impl MessageQueue for KafkaImpl { + }) + } + ++ // FIXME: consume a empty topic may be hanged forever... + async fn consume( + &self, + topic_name: &str, +diff --git a/components/message_queue/src/tests/cases.rs b/components/message_queue/src/tests/cases.rs +index 29ab6805..b268db61 100644 +--- a/components/message_queue/src/tests/cases.rs ++++ b/components/message_queue/src/tests/cases.rs +@@ -133,8 +133,7 @@ async fn test_consume_empty_topic(message_queue: &T) { + .await + .is_ok()); + +- // Call produce to push messages at first, then call consume to pull back and +- // compare. ++ // FIXME: consume a empty topic may be hanged forever... + let mut iter = message_queue + .consume(&topic_name, StartOffset::Earliest) + .await +diff --git a/wal/src/kv_encoder.rs b/wal/src/kv_encoder.rs +index e0e64fa4..c8e56802 100644 +--- a/wal/src/kv_encoder.rs ++++ b/wal/src/kv_encoder.rs +@@ -1,4 +1,4 @@ +-// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. ++// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. + + //! Common Encoding for Wal logs + +@@ -115,17 +115,15 @@ pub enum Namespace { + } + + /// Log key in old wal design, map the `TableId` to `RegionId` +-#[allow(unused)] ++ + pub type LogKey = (u64, SequenceNumber); + +-#[allow(unused)] + #[derive(Debug, Clone)] + pub struct LogKeyEncoder { + pub version: u8, + pub namespace: Namespace, + } + +-#[allow(unused)] + impl LogKeyEncoder { + /// Create newest version encoder. + pub fn newest() -> Self { +@@ -202,13 +200,11 @@ impl Decoder for LogKeyEncoder { + } + } + +-#[allow(unused)] + #[derive(Debug, Clone)] + pub struct LogValueEncoder { + pub version: u8, + } + +-#[allow(unused)] + impl LogValueEncoder { + /// Create newest version encoder. + pub fn newest() -> Self { +@@ -240,12 +236,10 @@ impl Encoder for LogValueEncoder { + } + } + +-#[allow(unused)] + pub struct LogValueDecoder { + pub version: u8, + } + +-#[allow(unused)] + impl LogValueDecoder { + pub fn decode<'a>(&self, mut buf: &'a [u8]) -> Result<&'a [u8]> { + let version = buf.try_get_u8().context(DecodeLogValueHeader)?; +@@ -475,7 +469,6 @@ impl MaxSeqMetaEncoding { + } + } + +-#[allow(unused)] + #[derive(Debug, Clone)] + pub struct LogEncoding { + key_enc: LogKeyEncoder, +@@ -484,7 +477,6 @@ pub struct LogEncoding { + value_enc_version: u8, + } + +-#[allow(unused)] + impl LogEncoding { + pub fn newest() -> Self { + Self { +@@ -589,7 +581,7 @@ impl LogBatchEncoder { + } + + /// Common log key used in multiple wal implementation +-#[allow(unused)] ++ + #[derive(Debug, Copy, Clone, Eq, PartialEq, PartialOrd, Ord)] + pub struct CommonLogKey { + /// Id of region which the table belongs to, +@@ -599,7 +591,6 @@ pub struct CommonLogKey { + pub sequence_num: SequenceNumber, + } + +-#[allow(unused)] + impl CommonLogKey { + pub fn new(region_id: u64, table_id: TableId, sequence_num: SequenceNumber) -> Self { + Self { +@@ -610,14 +601,12 @@ impl CommonLogKey { + } + } + +-#[allow(unused)] + #[derive(Debug, Clone)] + pub struct CommonLogKeyEncoder { + pub version: u8, + pub namespace: Namespace, + } + +-#[allow(unused)] + impl CommonLogKeyEncoder { + /// Create newest version encoder. + pub fn newest() -> Self { +@@ -697,7 +686,6 @@ impl Decoder for CommonLogKeyEncoder { + } + } + +-#[allow(unused)] + #[derive(Debug, Clone)] + pub struct CommonLogEncoding { + key_enc: CommonLogKeyEncoder, +@@ -706,7 +694,6 @@ pub struct CommonLogEncoding { + value_enc_version: u8, + } + +-#[allow(unused)] + impl CommonLogEncoding { + pub fn newest() -> Self { + Self { +diff --git a/wal/src/message_queue_impl/encoding.rs b/wal/src/message_queue_impl/encoding.rs +index 86139819..b392711b 100644 +--- a/wal/src/message_queue_impl/encoding.rs ++++ b/wal/src/message_queue_impl/encoding.rs +@@ -1,4 +1,4 @@ +-// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. ++// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. + + //! Meta encoding of wal's message queue implementation + +@@ -90,25 +90,23 @@ pub enum Error { + define_result!(Error); + + /// Generate wal data topic name +-#[allow(unused)] ++ + pub fn format_wal_data_topic_name(namespace: &str, region_id: u64) -> String { + format!("{namespace}_data_{region_id}") + } + + /// Generate wal meta topic name +-#[allow(unused)] ++ + pub fn format_wal_meta_topic_name(namespace: &str, region_id: u64) -> String { + format!("{namespace}_meta_{region_id}") + } + +-#[allow(unused)] + #[derive(Clone, Debug)] + pub struct MetaEncoding { + key_enc: MetaKeyEncoder, + value_enc: MetaValueEncoder, + } + +-#[allow(unused)] + impl MetaEncoding { + pub fn encode_key(&self, buf: &mut BytesMut, meta_key: &MetaKey) -> manager::Result<()> { + buf.clear(); +@@ -153,6 +151,7 @@ impl MetaEncoding { + Ok(meta_value.into()) + } + ++ #[allow(dead_code)] + pub fn is_meta_key(&self, mut buf: &[u8]) -> manager::Result { + self.key_enc + .is_valid(&mut buf) +@@ -174,20 +173,19 @@ impl MetaEncoding { + } + + /// Message queue implementation's meta key +-#[allow(unused)] ++ + #[derive(Clone, Debug, Eq, PartialEq)] + pub struct MetaKey(pub u64); + +-#[allow(unused)] + #[derive(Clone, Debug)] + pub struct MetaKeyEncoder { + pub namespace: Namespace, + pub version: u8, + } + +-#[allow(unused)] + impl MetaKeyEncoder { + /// Determine whether the raw bytes is a valid meta key. ++ #[allow(dead_code)] + pub fn is_valid(&self, buf: &mut B) -> Result { + let namespace = buf.try_get_u8().context(DecodeMetaKey)?; + let version = buf.try_get_u8().context(DecodeMetaKey)?; +diff --git a/wal/src/message_queue_impl/region.rs b/wal/src/message_queue_impl/region.rs +index 5ac07340..321c1978 100644 +--- a/wal/src/message_queue_impl/region.rs ++++ b/wal/src/message_queue_impl/region.rs +@@ -917,6 +917,7 @@ mod tests { + }; + + #[tokio::test] ++ #[ignore = "this test need a kafka cluster"] + async fn test_region_kafka_impl() { + // Test region + let mut config = Config::default(); +-- +2.32.1 (Apple Git-133) + + +From 75f9f8a0da1dab650e93b1fd7372c5d7d91c3069 Mon Sep 17 00:00:00 2001 +From: kamille +Date: Tue, 27 Jun 2023 15:10:15 +0800 +Subject: [PATCH 3/4] add logs to inspect wal on kafka. + +--- + wal/src/message_queue_impl/region_context.rs | 12 +++++++++++- + 1 file changed, 11 insertions(+), 1 deletion(-) + +diff --git a/wal/src/message_queue_impl/region_context.rs b/wal/src/message_queue_impl/region_context.rs +index 441eb93c..b035d15f 100644 +--- a/wal/src/message_queue_impl/region_context.rs ++++ b/wal/src/message_queue_impl/region_context.rs +@@ -13,7 +13,7 @@ use common_util::{ + define_result, + error::{BoxError, GenericError}, + }; +-use log::{debug, warn}; ++use log::{debug, info, warn}; + use message_queue::{MessageQueue, Offset}; + use snafu::{ensure, Backtrace, OptionExt, ResultExt, Snafu}; + use tokio::sync::{Mutex, RwLock}; +@@ -292,6 +292,11 @@ impl TableMeta { + ) -> std::result::Result<(), String> { + let mut inner = self.inner.lock().await; + ++ info!( ++ "Mark deleted entries to table begin, sequence_num:{sequence_num}, table_id:{}, mapping:{:?}", ++ self.table_id, inner.start_sequence_offset_mapping ++ ); ++ + // Check the set sequence num's validity. + if sequence_num > inner.next_sequence_num { + return Err(format!( +@@ -325,6 +330,11 @@ impl TableMeta { + .start_sequence_offset_mapping + .retain(|k, _| k >= &sequence_num); + ++ info!( ++ "Mark deleted entries to table finish, sequence_num:{sequence_num}, table_id:{}, mapping:{:?}", ++ self.table_id, inner.start_sequence_offset_mapping ++ ); ++ + Ok(()) + } + +-- +2.32.1 (Apple Git-133) + + +From 61d0df0eacc07ba90ac96d19843a4e82ee369463 Mon Sep 17 00:00:00 2001 +From: kamille +Date: Wed, 28 Jun 2023 11:33:17 +0800 +Subject: [PATCH 4/4] modify `apply_region_meta_delta` to adapt new recover + start sequence. + +--- + wal/src/message_queue_impl/region_context.rs | 11 ----------- + 1 file changed, 11 deletions(-) + +diff --git a/wal/src/message_queue_impl/region_context.rs b/wal/src/message_queue_impl/region_context.rs +index b035d15f..e2c1ffde 100644 +--- a/wal/src/message_queue_impl/region_context.rs ++++ b/wal/src/message_queue_impl/region_context.rs +@@ -517,18 +517,7 @@ impl RegionContextBuilder { + .entry(delta.table_id) + .or_insert_with(TableMetaInner::default); + +- ensure!(table_meta.next_sequence_num < delta.sequence_num + 1, Build { msg: format!("apply delta failed, +- next sequence number in delta should't be less than or equal to the one in builder, but now are:{} and {}", +- delta.sequence_num + 1, +- table_meta.next_sequence_num, +- ) }); + table_meta.next_sequence_num = delta.sequence_num + 1; +- +- ensure!(table_meta.current_high_watermark < delta.offset + 1, Build { msg: format!("apply delta failed, +- high watermark in delta should't be less than or equal to the one in builder, but now are:{} and {}", +- delta.offset + 1, +- table_meta.current_high_watermark, +- ) }); + table_meta.current_high_watermark = delta.offset + 1; + + // Because recover from the `region_safe_delete_offset`, some outdated logs will +-- +2.32.1 (Apple Git-133) + diff --git a/router/src/cluster_based.rs b/router/src/cluster_based.rs index a9e861631f..d72c56c40a 100644 --- a/router/src/cluster_based.rs +++ b/router/src/cluster_based.rs @@ -169,7 +169,9 @@ mod tests { }, storage::RequestContext, }; - use cluster::{shard_lock_manager::ShardLockManagerRef, Cluster, ClusterNodesResp}; + use cluster::{ + shard_lock_manager::ShardLockManagerRef, shard_set::ShardRef, Cluster, ClusterNodesResp, + }; use common_types::table::ShardId; use common_util::config::ReadableDuration; use meta_client::types::{ @@ -191,37 +193,15 @@ mod tests { unimplemented!(); } - async fn open_shard(&self, _: &ShardInfo) -> cluster::Result { - unimplemented!(); - } - - async fn close_shard(&self, _: ShardId) -> cluster::Result { + async fn open_shard(&self, _: &ShardInfo) -> cluster::Result { unimplemented!(); } - async fn freeze_shard(&self, _: ShardId) -> cluster::Result { - unimplemented!(); - } - - async fn create_table_on_shard( - &self, - _req: &CreateTableOnShardRequest, - ) -> cluster::Result<()> { + fn get_shard(&self, _: ShardId) -> Option { unimplemented!(); } - async fn drop_table_on_shard(&self, _req: &DropTableOnShardRequest) -> cluster::Result<()> { - unimplemented!(); - } - - async fn open_table_on_shard(&self, _req: &OpenTableOnShardRequest) -> cluster::Result<()> { - unimplemented!(); - } - - async fn close_table_on_shard( - &self, - _req: &CloseTableOnShardRequest, - ) -> cluster::Result<()> { + async fn close_shard(&self, _: ShardId) -> cluster::Result { unimplemented!(); } diff --git a/server/src/grpc/meta_event_service/mod.rs b/server/src/grpc/meta_event_service/mod.rs index fcabc3c8de..e74db09e0a 100644 --- a/server/src/grpc/meta_event_service/mod.rs +++ b/server/src/grpc/meta_event_service/mod.rs @@ -6,6 +6,7 @@ use std::{sync::Arc, time::Instant}; use analytic_engine::setup::OpenedWals; use async_trait::async_trait; +use snafu::OptionExt; use catalog::{ schema::{ CloseOptions, CloseTableRequest, CreateOptions, CreateTableRequest, DropOptions, @@ -23,9 +24,9 @@ use ceresdbproto::meta_event::{ }; use cluster::ClusterRef; use common_types::{schema::SchemaEncoder, table::ShardId}; -use common_util::{error::BoxError, runtime::Runtime, time::InstantExt}; +use common_util::{error::{BoxError, GenericError}, runtime::Runtime, time::InstantExt}; use log::{error, info, warn}; -use meta_client::types::ShardInfo; +use meta_client::types::{ShardInfo, TableInfo}; use paste::paste; use proxy::instance::InstanceRef; use query_engine::executor::Executor as QueryExecutor; @@ -247,9 +248,11 @@ impl HandlerContext { } async fn do_open_shard(ctx: HandlerContext, shard_info: ShardInfo) -> Result<()> { + // Try to lock the shard in node level. ctx.acquire_shard_lock(shard_info.id).await?; - let tables_of_shard = ctx + // Success to lock the shard in this node, fetch and open shard now. + let shard = ctx .cluster .open_shard(&shard_info) .await @@ -258,17 +261,19 @@ async fn do_open_shard(ctx: HandlerContext, shard_info: ShardInfo) -> Result<()> code: StatusCode::Internal, msg: "fail to open shards in cluster", })?; - + + // Lock the shard in local, and then recover it. + let shard = shard.write().await; let catalog_name = &ctx.default_catalog; - let shard_info = tables_of_shard.shard_info; - let table_defs = tables_of_shard + let shard_info = shard.shard_info.clone(); + let table_defs = shard .tables - .into_iter() + .iter() .map(|info| TableDef { catalog_name: catalog_name.clone(), - schema_name: info.schema_name, + schema_name: info.schema_name.clone(), id: TableId::from(info.id), - name: info.name, + name: info.name.clone(), }) .collect(); @@ -294,7 +299,6 @@ async fn do_open_shard(ctx: HandlerContext, shard_info: ShardInfo) -> Result<()> // TODO: maybe we should encapsulate the logic of handling meta event into a // trait, so that we don't need to expose the logic to the meta event service // implementation. - async fn handle_open_shard(ctx: HandlerContext, request: OpenShardRequest) -> Result<()> { info!("Receive open shard request, request:{request:?}"); let shard_info = ShardInfo::from(&request.shard.context(ErrNoCause { @@ -493,7 +497,8 @@ async fn handle_drop_table_on_shard( ctx: HandlerContext, request: DropTableOnShardRequest, ) -> Result<()> { - ctx.cluster + let shard = ctx + .cluster .drop_table_on_shard(&request) .await .box_err() @@ -590,16 +595,16 @@ async fn handle_close_table_on_shard( ctx: HandlerContext, request: CloseTableOnShardRequest, ) -> Result<()> { - ctx.cluster - .close_table_on_shard(&request) - .await - .box_err() - .with_context(|| ErrWithCause { + let shard = ctx.cluster + .get_shard(shard_id) + .with_context(|| ErrNoCause { code: StatusCode::Internal, - msg: format!("fail to close table on shard in cluster, req:{request:?}"), + msg: format!("fail to close table on shard in cluster, req:{:?}", request.clone()), })?; - // Close the table by catalog manager afterwards. + let shard = shard.write().await; + + // Close the table by catalog manager afterwards. let catalog_name = &ctx.default_catalog; let table_info = request.table_info.context(ErrNoCause { code: StatusCode::BadRequest, @@ -629,6 +634,20 @@ async fn handle_close_table_on_shard( Ok(()) } +struct UpdateShardInfo { + prev_version: u64, + shard_info: ShardInfo, + table_info: TableInfo, +} + +impl TryFrom for UpdateShardInfo { + type Error = GenericError; + + fn try_from(value: ceresdbproto::meta_event::UpdateShardInfo) -> std::result::Result { + todo!() + } +} + #[async_trait] impl MetaEventService for MetaServiceImpl { async fn open_shard( From 783113fbe08e929e59eb3dba11aceb52aee9acb2 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 7 Jul 2023 18:18:33 +0800 Subject: [PATCH 05/15] modify shard request processing logic based on the new `Cluster` and `Shard` interface. --- ...fix-bug-about-empty-region-s-opening.patch | 99 --- cluster/src/shard_set.rs | 39 +- kafka-delete.patch | 622 ------------------ server/src/grpc/meta_event_service/mod.rs | 203 ++++-- src/setup.rs | 2 +- 5 files changed, 172 insertions(+), 793 deletions(-) delete mode 100644 0001-fix-bug-about-empty-region-s-opening.patch delete mode 100644 kafka-delete.patch diff --git a/0001-fix-bug-about-empty-region-s-opening.patch b/0001-fix-bug-about-empty-region-s-opening.patch deleted file mode 100644 index 2712bc9660..0000000000 --- a/0001-fix-bug-about-empty-region-s-opening.patch +++ /dev/null @@ -1,99 +0,0 @@ -From bfcc7c79f6d3a7a92ec720358742880fd9a54147 Mon Sep 17 00:00:00 2001 -From: kamille -Date: Fri, 30 Jun 2023 02:23:45 +0800 -Subject: [PATCH] fix bug about empty region's opening. - ---- - wal/src/message_queue_impl/region.rs | 48 +++++++++++++++++++++++++--- - 1 file changed, 43 insertions(+), 5 deletions(-) - -diff --git a/wal/src/message_queue_impl/region.rs b/wal/src/message_queue_impl/region.rs -index 321c1978..d1995387 100644 ---- a/wal/src/message_queue_impl/region.rs -+++ b/wal/src/message_queue_impl/region.rs -@@ -224,7 +224,17 @@ impl Region { - namespace, region_id - ); - -- // Fetch high watermark and check. -+ // Fetch earliest, high watermark and check. -+ let earliest = message_queue -+ .fetch_offset(meta_topic, OffsetType::EarliestOffset) -+ .await -+ .box_err() -+ .context(OpenWithCause { -+ namespace, -+ region_id, -+ msg: "failed while recover from meta", -+ })?; -+ - let high_watermark = message_queue - .fetch_offset(meta_topic, OffsetType::HighWaterMark) - .await -@@ -235,9 +245,19 @@ impl Region { - msg: "failed while recover from meta", - })?; - -- if high_watermark == 0 { -- debug!("Meta topic is empty, it just needs to recover from log topic, namespace:{}, region id:{}", namespace, region_id); -- return Ok(None); -+ if earliest == high_watermark { -+ if high_watermark == 0 { -+ info!("Recover region meta from meta, found empty meta topic, just need to recover from log topic, namespace:{}, region id:{}", -+ namespace, region_id); -+ return Ok(None); -+ } -+ -+ return OpenNoCause { -+ namespace, -+ region_id, -+ msg: "region meta impossible to be empty when having written logs", -+ } -+ .fail(); - } - - // Fetch snapshot from meta topic(just fetch the last snapshot). -@@ -313,6 +333,9 @@ impl Region { - }); - - let region_safe_delete_offset = if min_safe_delete_offset == i64::MAX { -+ info!("Recover region meta from meta, min_safe_delete_offset not exist, region_meta_snapshot:{:?}, namespace:{}, region id:{}", -+ value, namespace, region_id); -+ - None - } else { - Some(min_safe_delete_offset) -@@ -353,6 +376,17 @@ impl Region { - // FIXME: should not judge whether topic is empty or not by caller. - // The consumer iterator should return immediately rather than hanging when - // topic empty. -+ // Fetch earliest, high watermark and check. -+ let earliest = message_queue -+ .fetch_offset(log_topic, OffsetType::EarliestOffset) -+ .await -+ .box_err() -+ .context(OpenWithCause { -+ namespace, -+ region_id, -+ msg: "failed while recover from log", -+ })?; -+ - let high_watermark = message_queue - .fetch_offset(log_topic, OffsetType::HighWaterMark) - .await -@@ -362,7 +396,11 @@ impl Region { - region_id, - msg: "failed while recover from log", - })?; -- if high_watermark == 0 { -+ -+ if earliest == high_watermark { -+ info!("Recover region meta from log, found empty log topic, namespace:{}, region_id:{}, earliest:{}, high_watermark:{}", -+ namespace, region_id, earliest, high_watermark -+ ); - return Ok(()); - } - --- -2.32.1 (Apple Git-133) - diff --git a/cluster/src/shard_set.rs b/cluster/src/shard_set.rs index 0a6e3fcbb2..2a7f5928a3 100644 --- a/cluster/src/shard_set.rs +++ b/cluster/src/shard_set.rs @@ -8,10 +8,7 @@ use std::{ use meta_client::types::{ShardId, ShardInfo, ShardVersion, TableInfo, TablesOfShard}; use snafu::{ensure, OptionExt}; -use crate::{ - Result, ShardNotFound, ShardVersionMismatch, TableAlreadyExists, TableNotFound, - UpdateFrozenShard, -}; +use crate::{Result, ShardVersionMismatch, TableAlreadyExists, TableNotFound, UpdateFrozenShard}; /// [ShardTablesCache] caches the information about tables and shards, and the /// relationship between them is: one shard -> multiple tables. @@ -26,6 +23,12 @@ pub struct TableWithShards { pub shard_infos: Vec, } +pub struct UpdatedTableInfo { + pub prev_version: u64, + pub shard_info: ShardInfo, + pub table_info: TableInfo, +} + impl ShardSet { // Fetch all the shard infos. pub fn all_shards(&self) -> Vec { @@ -61,16 +64,17 @@ pub struct Shard { } impl Shard { - fn freeze(&mut self) { + pub fn freeze(&mut self) { self.frozen = true; } - fn try_insert_table( - &mut self, - prev_shard_version: ShardVersion, - curr_shard: ShardInfo, - new_table: TableInfo, - ) -> Result<()> { + pub fn try_insert_table(&mut self, updated_info: UpdatedTableInfo) -> Result<()> { + let UpdatedTableInfo { + prev_version: prev_shard_version, + shard_info: curr_shard, + table_info: new_table, + } = updated_info; + ensure!( !self.frozen, UpdateFrozenShard { @@ -101,12 +105,13 @@ impl Shard { Ok(()) } - fn try_remove_table( - &mut self, - prev_shard_version: ShardVersion, - curr_shard: ShardInfo, - new_table: TableInfo, - ) -> Result<()> { + pub fn try_remove_table(&mut self, updated_info: UpdatedTableInfo) -> Result<()> { + let UpdatedTableInfo { + prev_version: prev_shard_version, + shard_info: curr_shard, + table_info: new_table, + } = updated_info; + ensure!( !self.frozen, UpdateFrozenShard { diff --git a/kafka-delete.patch b/kafka-delete.patch deleted file mode 100644 index 97a587993c..0000000000 --- a/kafka-delete.patch +++ /dev/null @@ -1,622 +0,0 @@ -From dee7d12ee23245efdd08efd4fd8f9b18ddc22e4b Mon Sep 17 00:00:00 2001 -From: kamille -Date: Tue, 27 Jun 2023 14:27:53 +0800 -Subject: [PATCH 1/4] fix deleted sequence updating and recover begin offset - computing. - ---- - wal/src/message_queue_impl/region.rs | 71 +++++++++++--------- - wal/src/message_queue_impl/region_context.rs | 19 +++--- - 2 files changed, 50 insertions(+), 40 deletions(-) - -diff --git a/wal/src/message_queue_impl/region.rs b/wal/src/message_queue_impl/region.rs -index 37beea03..5ac07340 100644 ---- a/wal/src/message_queue_impl/region.rs -+++ b/wal/src/message_queue_impl/region.rs -@@ -148,6 +148,7 @@ impl Region { - region_id, - msg: "failed while trying to create topic", - })?; -+ - message_queue - .create_topic_if_not_exist(&meta_topic) - .await -@@ -160,7 +161,7 @@ impl Region { - - // Build region meta. - let mut region_meta_builder = RegionContextBuilder::new(region_id); -- let high_watermark_in_snapshot = Self::recover_region_meta_from_meta( -+ let region_safe_delete_offset = Self::recover_region_meta_from_meta( - namespace, - region_id, - message_queue.as_ref(), -@@ -174,7 +175,7 @@ impl Region { - namespace, - region_id, - message_queue.as_ref(), -- high_watermark_in_snapshot, -+ region_safe_delete_offset, - &log_topic, - &log_encoding, - &mut region_meta_builder, -@@ -217,7 +218,7 @@ impl Region { - meta_topic: &str, - meta_encoding: &MetaEncoding, - builder: &mut RegionContextBuilder, -- ) -> Result { -+ ) -> Result> { - info!( - "Recover region meta from meta, namespace:{}, region id:{}", - namespace, region_id -@@ -236,7 +237,7 @@ impl Region { - - if high_watermark == 0 { - debug!("Meta topic is empty, it just needs to recover from log topic, namespace:{}, region id:{}", namespace, region_id); -- return Ok(0); -+ return Ok(None); - } - - // Fetch snapshot from meta topic(just fetch the last snapshot). -@@ -304,10 +305,18 @@ impl Region { - msg: "failed while recover from meta", - })?; - -- let high_watermark_in_snapshot = value -- .entries -- .iter() -- .fold(0, |hw, entry| cmp::max(hw, entry.current_high_watermark)); -+ let min_safe_delete_offset = value.entries.iter().fold(i64::MAX, |min_offset, entry| { -+ match entry.safe_delete_offset { -+ Some(offset) => cmp::min(min_offset, offset), -+ None => min_offset, -+ } -+ }); -+ -+ let region_safe_delete_offset = if min_safe_delete_offset == i64::MAX { -+ None -+ } else { -+ Some(min_safe_delete_offset) -+ }; - - builder - .apply_region_meta_snapshot(value) -@@ -318,24 +327,32 @@ impl Region { - msg: "failed while recover from meta", - })?; - -- Ok(high_watermark_in_snapshot) -+ Ok(region_safe_delete_offset) - } - - async fn recover_region_meta_from_log( - namespace: &str, - region_id: u64, - message_queue: &M, -- start_offset: Offset, -+ region_safe_delete_offset: Option, - log_topic: &str, - log_encoding: &CommonLogEncoding, - builder: &mut RegionContextBuilder, - ) -> Result<()> { - info!( -- "Recover region meta from log, namespace:{}, region id:{}, start offset:{}", -- namespace, region_id, start_offset -+ "Recover region meta from log, namespace:{}, region_id:{}, region_safe_delete_offset:{:?}", -+ namespace, region_id, region_safe_delete_offset - ); - -- // Fetch high watermark and check. -+ let start_offset = match region_safe_delete_offset { -+ Some(offset) => StartOffset::At(offset), -+ None => StartOffset::Earliest, -+ }; -+ -+ // Fetch snapshot from meta topic(just fetch the last snapshot). -+ // FIXME: should not judge whether topic is empty or not by caller. -+ // The consumer iterator should return immediately rather than hanging when -+ // topic empty. - let high_watermark = message_queue - .fetch_offset(log_topic, OffsetType::HighWaterMark) - .await -@@ -345,19 +362,12 @@ impl Region { - region_id, - msg: "failed while recover from log", - })?; -- -- ensure!(start_offset <= high_watermark, OpenNoCause { namespace , region_id, msg: format!( -- "failed while recover from log, start offset should be less than or equal to high watermark, now are:{start_offset} and {high_watermark}") -- }); -- -- if start_offset == high_watermark { -- debug!("No region meta delta from log topic is needed, just return, namespace:{}, region id:{}", namespace, region_id); -+ if high_watermark == 0 { - return Ok(()); - } - -- // Fetch snapshot from meta topic(just fetch the last snapshot). - let mut iter = message_queue -- .consume(log_topic, StartOffset::At(start_offset)) -+ .consume(log_topic, start_offset) - .await - .box_err() - .context(OpenWithCause { -@@ -637,7 +647,7 @@ impl Region { - } - - /// Return snapshot, just used for test. -- #[allow(unused)] -+ #[allow(dead_code)] - async fn make_meta_snapshot(&self) -> RegionMetaSnapshot { - let inner = self.inner.write().await; - inner.make_meta_snapshot().await -@@ -907,7 +917,6 @@ mod tests { - }; - - #[tokio::test] -- #[ignore] - async fn test_region_kafka_impl() { - // Test region - let mut config = Config::default(); -@@ -962,7 +971,7 @@ mod tests { - .write(&WriteContext::default(), test_log_batch) - .await - .unwrap(); -- assert_eq!(sequence_num, test_log_batch.len() as u64 - 1); -+ assert_eq!(sequence_num, test_log_batch.len() as u64); - - mixed_test_payloads.extend_from_slice(test_payloads); - } -@@ -1079,18 +1088,18 @@ mod tests { - .write(&WriteContext::default(), test_log_batch) - .await - .unwrap(); -- assert_eq!(sequence_num, test_log_batch.len() as u64 - 1); -+ assert_eq!(sequence_num, test_log_batch.len() as u64); - let sequence_num = test_context - .region - .write(&WriteContext::default(), test_log_batch) - .await - .unwrap(); -- assert_eq!(sequence_num, test_log_batch.len() as u64 * 2 - 1); -+ assert_eq!(sequence_num, test_log_batch.len() as u64 * 2); - - // Mark deleted. - test_context - .region -- .mark_delete_to(table_id, test_log_batch.len() as u64) -+ .mark_delete_to(table_id, test_log_batch.len() as u64 + 1) - .await - .unwrap(); - let table_meta = test_context -@@ -1101,11 +1110,11 @@ mod tests { - .unwrap(); - assert_eq!( - table_meta.next_sequence_num, -- test_log_batch.len() as u64 * 2 -+ test_log_batch.len() as u64 * 2 + 1, - ); - assert_eq!( - table_meta.latest_marked_deleted, -- test_log_batch.len() as u64 -+ test_log_batch.len() as u64 + 1 - ); - assert_eq!( - table_meta.current_high_watermark, -@@ -1151,7 +1160,7 @@ mod tests { - .write(&WriteContext::default(), test_log_batch) - .await - .unwrap(); -- assert_eq!(sequence_num, test_log_batch.len() as u64 - 1); -+ assert_eq!(sequence_num, test_log_batch.len() as u64); - - mixed_test_payloads.extend_from_slice(test_payloads); - } -diff --git a/wal/src/message_queue_impl/region_context.rs b/wal/src/message_queue_impl/region_context.rs -index 3b611e59..441eb93c 100644 ---- a/wal/src/message_queue_impl/region_context.rs -+++ b/wal/src/message_queue_impl/region_context.rs -@@ -1,4 +1,4 @@ --// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. -+// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. - - //! Region context - -@@ -310,7 +310,7 @@ impl TableMeta { - // The `start_sequence_offset_mapping` is possible to be incomplete during - // recovery. - let offset = inner.start_sequence_offset_mapping.get(&sequence_num); -- if offset.is_none() && inner.next_sequence_num != inner.latest_marked_deleted { -+ if offset.is_none() && inner.next_sequence_num != sequence_num { - warn!("Start sequence offset mapping is incomplete, - just not update the marked deleted sequence in this flush, new marked deleted, sequence num:{}, previous:{}", - sequence_num, inner.latest_marked_deleted); -@@ -469,14 +469,12 @@ impl OffsetRange { - } - - /// Builder for `RegionMeta` --#[allow(unused)] - #[derive(Debug)] - pub struct RegionContextBuilder { - region_id: u64, - table_metas: HashMap, - } - --#[allow(unused)] - impl RegionContextBuilder { - pub fn new(region_id: u64) -> Self { - Self { -@@ -503,6 +501,7 @@ impl RegionContextBuilder { - pub fn apply_region_meta_delta(&mut self, delta: RegionMetaDelta) -> Result<()> { - debug!("Apply region meta delta, delta:{:?}", delta); - -+ // It is likely that snapshot not exist(e.g. no table has ever flushed). - let mut table_meta = self - .table_metas - .entry(delta.table_id) -@@ -522,9 +521,13 @@ impl RegionContextBuilder { - ) }); - table_meta.current_high_watermark = delta.offset + 1; - -- table_meta -- .start_sequence_offset_mapping -- .insert(delta.sequence_num, delta.offset); -+ // Because recover from the `region_safe_delete_offset`, some outdated logs will -+ // be loaded. -+ if delta.sequence_num >= table_meta.latest_marked_deleted { -+ table_meta -+ .start_sequence_offset_mapping -+ .insert(delta.sequence_num, delta.offset); -+ } - - Ok(()) - } -@@ -557,7 +560,6 @@ impl RegionContextBuilder { - } - } - --#[allow(unused)] - #[derive(Debug, Clone)] - pub struct RegionMetaDelta { - table_id: TableId, -@@ -565,7 +567,6 @@ pub struct RegionMetaDelta { - offset: Offset, - } - --#[allow(unused)] - impl RegionMetaDelta { - pub fn new(table_id: TableId, sequence_num: SequenceNumber, offset: Offset) -> Self { - Self { --- -2.32.1 (Apple Git-133) - - -From 17fefa5de7ad20a3b262285f7432a3430d75520b Mon Sep 17 00:00:00 2001 -From: kamille -Date: Tue, 27 Jun 2023 14:39:56 +0800 -Subject: [PATCH 2/4] fix kafka related tests and clippy. - ---- - .../message_queue/src/kafka/kafka_impl.rs | 1 + - components/message_queue/src/tests/cases.rs | 3 +-- - wal/src/kv_encoder.rs | 19 +++---------------- - wal/src/message_queue_impl/encoding.rs | 14 ++++++-------- - wal/src/message_queue_impl/region.rs | 1 + - 5 files changed, 12 insertions(+), 26 deletions(-) - -diff --git a/components/message_queue/src/kafka/kafka_impl.rs b/components/message_queue/src/kafka/kafka_impl.rs -index 0d69d5e3..0fdfb462 100644 ---- a/components/message_queue/src/kafka/kafka_impl.rs -+++ b/components/message_queue/src/kafka/kafka_impl.rs -@@ -284,6 +284,7 @@ impl MessageQueue for KafkaImpl { - }) - } - -+ // FIXME: consume a empty topic may be hanged forever... - async fn consume( - &self, - topic_name: &str, -diff --git a/components/message_queue/src/tests/cases.rs b/components/message_queue/src/tests/cases.rs -index 29ab6805..b268db61 100644 ---- a/components/message_queue/src/tests/cases.rs -+++ b/components/message_queue/src/tests/cases.rs -@@ -133,8 +133,7 @@ async fn test_consume_empty_topic(message_queue: &T) { - .await - .is_ok()); - -- // Call produce to push messages at first, then call consume to pull back and -- // compare. -+ // FIXME: consume a empty topic may be hanged forever... - let mut iter = message_queue - .consume(&topic_name, StartOffset::Earliest) - .await -diff --git a/wal/src/kv_encoder.rs b/wal/src/kv_encoder.rs -index e0e64fa4..c8e56802 100644 ---- a/wal/src/kv_encoder.rs -+++ b/wal/src/kv_encoder.rs -@@ -1,4 +1,4 @@ --// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. -+// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. - - //! Common Encoding for Wal logs - -@@ -115,17 +115,15 @@ pub enum Namespace { - } - - /// Log key in old wal design, map the `TableId` to `RegionId` --#[allow(unused)] -+ - pub type LogKey = (u64, SequenceNumber); - --#[allow(unused)] - #[derive(Debug, Clone)] - pub struct LogKeyEncoder { - pub version: u8, - pub namespace: Namespace, - } - --#[allow(unused)] - impl LogKeyEncoder { - /// Create newest version encoder. - pub fn newest() -> Self { -@@ -202,13 +200,11 @@ impl Decoder for LogKeyEncoder { - } - } - --#[allow(unused)] - #[derive(Debug, Clone)] - pub struct LogValueEncoder { - pub version: u8, - } - --#[allow(unused)] - impl LogValueEncoder { - /// Create newest version encoder. - pub fn newest() -> Self { -@@ -240,12 +236,10 @@ impl Encoder for LogValueEncoder { - } - } - --#[allow(unused)] - pub struct LogValueDecoder { - pub version: u8, - } - --#[allow(unused)] - impl LogValueDecoder { - pub fn decode<'a>(&self, mut buf: &'a [u8]) -> Result<&'a [u8]> { - let version = buf.try_get_u8().context(DecodeLogValueHeader)?; -@@ -475,7 +469,6 @@ impl MaxSeqMetaEncoding { - } - } - --#[allow(unused)] - #[derive(Debug, Clone)] - pub struct LogEncoding { - key_enc: LogKeyEncoder, -@@ -484,7 +477,6 @@ pub struct LogEncoding { - value_enc_version: u8, - } - --#[allow(unused)] - impl LogEncoding { - pub fn newest() -> Self { - Self { -@@ -589,7 +581,7 @@ impl LogBatchEncoder { - } - - /// Common log key used in multiple wal implementation --#[allow(unused)] -+ - #[derive(Debug, Copy, Clone, Eq, PartialEq, PartialOrd, Ord)] - pub struct CommonLogKey { - /// Id of region which the table belongs to, -@@ -599,7 +591,6 @@ pub struct CommonLogKey { - pub sequence_num: SequenceNumber, - } - --#[allow(unused)] - impl CommonLogKey { - pub fn new(region_id: u64, table_id: TableId, sequence_num: SequenceNumber) -> Self { - Self { -@@ -610,14 +601,12 @@ impl CommonLogKey { - } - } - --#[allow(unused)] - #[derive(Debug, Clone)] - pub struct CommonLogKeyEncoder { - pub version: u8, - pub namespace: Namespace, - } - --#[allow(unused)] - impl CommonLogKeyEncoder { - /// Create newest version encoder. - pub fn newest() -> Self { -@@ -697,7 +686,6 @@ impl Decoder for CommonLogKeyEncoder { - } - } - --#[allow(unused)] - #[derive(Debug, Clone)] - pub struct CommonLogEncoding { - key_enc: CommonLogKeyEncoder, -@@ -706,7 +694,6 @@ pub struct CommonLogEncoding { - value_enc_version: u8, - } - --#[allow(unused)] - impl CommonLogEncoding { - pub fn newest() -> Self { - Self { -diff --git a/wal/src/message_queue_impl/encoding.rs b/wal/src/message_queue_impl/encoding.rs -index 86139819..b392711b 100644 ---- a/wal/src/message_queue_impl/encoding.rs -+++ b/wal/src/message_queue_impl/encoding.rs -@@ -1,4 +1,4 @@ --// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0. -+// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. - - //! Meta encoding of wal's message queue implementation - -@@ -90,25 +90,23 @@ pub enum Error { - define_result!(Error); - - /// Generate wal data topic name --#[allow(unused)] -+ - pub fn format_wal_data_topic_name(namespace: &str, region_id: u64) -> String { - format!("{namespace}_data_{region_id}") - } - - /// Generate wal meta topic name --#[allow(unused)] -+ - pub fn format_wal_meta_topic_name(namespace: &str, region_id: u64) -> String { - format!("{namespace}_meta_{region_id}") - } - --#[allow(unused)] - #[derive(Clone, Debug)] - pub struct MetaEncoding { - key_enc: MetaKeyEncoder, - value_enc: MetaValueEncoder, - } - --#[allow(unused)] - impl MetaEncoding { - pub fn encode_key(&self, buf: &mut BytesMut, meta_key: &MetaKey) -> manager::Result<()> { - buf.clear(); -@@ -153,6 +151,7 @@ impl MetaEncoding { - Ok(meta_value.into()) - } - -+ #[allow(dead_code)] - pub fn is_meta_key(&self, mut buf: &[u8]) -> manager::Result { - self.key_enc - .is_valid(&mut buf) -@@ -174,20 +173,19 @@ impl MetaEncoding { - } - - /// Message queue implementation's meta key --#[allow(unused)] -+ - #[derive(Clone, Debug, Eq, PartialEq)] - pub struct MetaKey(pub u64); - --#[allow(unused)] - #[derive(Clone, Debug)] - pub struct MetaKeyEncoder { - pub namespace: Namespace, - pub version: u8, - } - --#[allow(unused)] - impl MetaKeyEncoder { - /// Determine whether the raw bytes is a valid meta key. -+ #[allow(dead_code)] - pub fn is_valid(&self, buf: &mut B) -> Result { - let namespace = buf.try_get_u8().context(DecodeMetaKey)?; - let version = buf.try_get_u8().context(DecodeMetaKey)?; -diff --git a/wal/src/message_queue_impl/region.rs b/wal/src/message_queue_impl/region.rs -index 5ac07340..321c1978 100644 ---- a/wal/src/message_queue_impl/region.rs -+++ b/wal/src/message_queue_impl/region.rs -@@ -917,6 +917,7 @@ mod tests { - }; - - #[tokio::test] -+ #[ignore = "this test need a kafka cluster"] - async fn test_region_kafka_impl() { - // Test region - let mut config = Config::default(); --- -2.32.1 (Apple Git-133) - - -From 75f9f8a0da1dab650e93b1fd7372c5d7d91c3069 Mon Sep 17 00:00:00 2001 -From: kamille -Date: Tue, 27 Jun 2023 15:10:15 +0800 -Subject: [PATCH 3/4] add logs to inspect wal on kafka. - ---- - wal/src/message_queue_impl/region_context.rs | 12 +++++++++++- - 1 file changed, 11 insertions(+), 1 deletion(-) - -diff --git a/wal/src/message_queue_impl/region_context.rs b/wal/src/message_queue_impl/region_context.rs -index 441eb93c..b035d15f 100644 ---- a/wal/src/message_queue_impl/region_context.rs -+++ b/wal/src/message_queue_impl/region_context.rs -@@ -13,7 +13,7 @@ use common_util::{ - define_result, - error::{BoxError, GenericError}, - }; --use log::{debug, warn}; -+use log::{debug, info, warn}; - use message_queue::{MessageQueue, Offset}; - use snafu::{ensure, Backtrace, OptionExt, ResultExt, Snafu}; - use tokio::sync::{Mutex, RwLock}; -@@ -292,6 +292,11 @@ impl TableMeta { - ) -> std::result::Result<(), String> { - let mut inner = self.inner.lock().await; - -+ info!( -+ "Mark deleted entries to table begin, sequence_num:{sequence_num}, table_id:{}, mapping:{:?}", -+ self.table_id, inner.start_sequence_offset_mapping -+ ); -+ - // Check the set sequence num's validity. - if sequence_num > inner.next_sequence_num { - return Err(format!( -@@ -325,6 +330,11 @@ impl TableMeta { - .start_sequence_offset_mapping - .retain(|k, _| k >= &sequence_num); - -+ info!( -+ "Mark deleted entries to table finish, sequence_num:{sequence_num}, table_id:{}, mapping:{:?}", -+ self.table_id, inner.start_sequence_offset_mapping -+ ); -+ - Ok(()) - } - --- -2.32.1 (Apple Git-133) - - -From 61d0df0eacc07ba90ac96d19843a4e82ee369463 Mon Sep 17 00:00:00 2001 -From: kamille -Date: Wed, 28 Jun 2023 11:33:17 +0800 -Subject: [PATCH 4/4] modify `apply_region_meta_delta` to adapt new recover - start sequence. - ---- - wal/src/message_queue_impl/region_context.rs | 11 ----------- - 1 file changed, 11 deletions(-) - -diff --git a/wal/src/message_queue_impl/region_context.rs b/wal/src/message_queue_impl/region_context.rs -index b035d15f..e2c1ffde 100644 ---- a/wal/src/message_queue_impl/region_context.rs -+++ b/wal/src/message_queue_impl/region_context.rs -@@ -517,18 +517,7 @@ impl RegionContextBuilder { - .entry(delta.table_id) - .or_insert_with(TableMetaInner::default); - -- ensure!(table_meta.next_sequence_num < delta.sequence_num + 1, Build { msg: format!("apply delta failed, -- next sequence number in delta should't be less than or equal to the one in builder, but now are:{} and {}", -- delta.sequence_num + 1, -- table_meta.next_sequence_num, -- ) }); - table_meta.next_sequence_num = delta.sequence_num + 1; -- -- ensure!(table_meta.current_high_watermark < delta.offset + 1, Build { msg: format!("apply delta failed, -- high watermark in delta should't be less than or equal to the one in builder, but now are:{} and {}", -- delta.offset + 1, -- table_meta.current_high_watermark, -- ) }); - table_meta.current_high_watermark = delta.offset + 1; - - // Because recover from the `region_safe_delete_offset`, some outdated logs will --- -2.32.1 (Apple Git-133) - diff --git a/server/src/grpc/meta_event_service/mod.rs b/server/src/grpc/meta_event_service/mod.rs index e74db09e0a..9a2a9a4efd 100644 --- a/server/src/grpc/meta_event_service/mod.rs +++ b/server/src/grpc/meta_event_service/mod.rs @@ -6,7 +6,6 @@ use std::{sync::Arc, time::Instant}; use analytic_engine::setup::OpenedWals; use async_trait::async_trait; -use snafu::OptionExt; use catalog::{ schema::{ CloseOptions, CloseTableRequest, CreateOptions, CreateTableRequest, DropOptions, @@ -15,16 +14,20 @@ use catalog::{ table_operator::TableOperator, }; use ceresdbproto::meta_event::{ - meta_event_service_server::MetaEventService, ChangeShardRoleRequest, ChangeShardRoleResponse, - CloseShardRequest, CloseShardResponse, CloseTableOnShardRequest, CloseTableOnShardResponse, - CreateTableOnShardRequest, CreateTableOnShardResponse, DropTableOnShardRequest, - DropTableOnShardResponse, MergeShardsRequest, MergeShardsResponse, OpenShardRequest, - OpenShardResponse, OpenTableOnShardRequest, OpenTableOnShardResponse, SplitShardRequest, - SplitShardResponse, + self, meta_event_service_server::MetaEventService, ChangeShardRoleRequest, + ChangeShardRoleResponse, CloseShardRequest, CloseShardResponse, CloseTableOnShardRequest, + CloseTableOnShardResponse, CreateTableOnShardRequest, CreateTableOnShardResponse, + DropTableOnShardRequest, DropTableOnShardResponse, MergeShardsRequest, MergeShardsResponse, + OpenShardRequest, OpenShardResponse, OpenTableOnShardRequest, OpenTableOnShardResponse, + SplitShardRequest, SplitShardResponse, }; -use cluster::ClusterRef; +use cluster::{shard_set::UpdatedTableInfo, ClusterRef}; use common_types::{schema::SchemaEncoder, table::ShardId}; -use common_util::{error::{BoxError, GenericError}, runtime::Runtime, time::InstantExt}; +use common_util::{ + error::{BoxError, GenericError}, + runtime::Runtime, + time::InstantExt, +}; use log::{error, info, warn}; use meta_client::types::{ShardInfo, TableInfo}; use paste::paste; @@ -51,6 +54,41 @@ use crate::grpc::{ mod error; mod shard_operation; +macro_rules! extract_updated_table_info { + ($request: expr) => {{ + let update_shard_info = $request.update_shard_info.clone(); + let table_info = $request.table_info.clone(); + + let update_shard_info = update_shard_info.context(ErrNoCause { + code: StatusCode::Internal, + msg: "update shard info is missing", + })?; + let curr_shard_info = update_shard_info.curr_shard_info.context(ErrNoCause { + code: StatusCode::Internal, + msg: "update shard info is missing", + })?; + let table_info = table_info.context(ErrNoCause { + code: StatusCode::Internal, + msg: "update shard info is missing", + })?; + + let prev_version = update_shard_info.prev_version; + let shard_info = ShardInfo::from(&curr_shard_info); + let table_info = TableInfo::try_from(table_info) + .box_err() + .context(ErrWithCause { + code: StatusCode::Internal, + msg: "failed to parse tableInfo", + })?; + + UpdatedTableInfo { + prev_version, + shard_info, + table_info, + } + }}; +} + /// Builder for [MetaServiceImpl]. pub struct Builder { pub cluster: ClusterRef, @@ -261,9 +299,10 @@ async fn do_open_shard(ctx: HandlerContext, shard_info: ShardInfo) -> Result<()> code: StatusCode::Internal, msg: "fail to open shards in cluster", })?; - + // Lock the shard in local, and then recover it. let shard = shard.write().await; + let catalog_name = &ctx.default_catalog; let shard_info = shard.shard_info.clone(); let table_defs = shard @@ -321,27 +360,30 @@ async fn handle_open_shard(ctx: HandlerContext, request: OpenShardRequest) -> Re } async fn do_close_shard(ctx: &HandlerContext, shard_id: ShardId) -> Result<()> { - let tables_of_shard = - ctx.cluster - .freeze_shard(shard_id) - .await - .box_err() - .context(ErrWithCause { - code: StatusCode::Internal, - msg: "fail to freeze shard before close it in cluster", - })?; + let shard = ctx + .cluster + .get_shard(shard_id) + .with_context(|| ErrNoCause { + code: StatusCode::Internal, + msg: format!("shard not found when closing shard, shard_id:{shard_id}",), + })?; + + let mut shard = shard.write().await; + + shard.freeze(); + info!("Shard is frozen before closed, shard_id:{shard_id}"); let catalog_name = &ctx.default_catalog.clone(); - let shard_info = tables_of_shard.shard_info; - let table_defs = tables_of_shard + let shard_info = shard.shard_info.clone(); + let table_defs = shard .tables - .into_iter() + .iter() .map(|info| TableDef { catalog_name: catalog_name.clone(), - schema_name: info.schema_name, + schema_name: info.schema_name.clone(), id: TableId::from(info.id), - name: info.name, + name: info.name.clone(), }) .collect(); let close_shard_request = catalog::schema::CloseShardRequest { @@ -409,13 +451,30 @@ async fn handle_create_table_on_shard( ctx: HandlerContext, request: CreateTableOnShardRequest, ) -> Result<()> { - ctx.cluster - .create_table_on_shard(&request) - .await + let updated_table_info = extract_updated_table_info!(request); + let shard = ctx + .cluster + .get_shard(updated_table_info.shard_info.id) + .with_context(|| ErrNoCause { + code: StatusCode::Internal, + msg: format!( + "shard not found when creating table on shard, req:{:?}", + request.clone() + ), + })?; + + let mut shard = shard.write().await; + + // FIXME: should insert table from cluster after having created table. + shard + .try_insert_table(updated_table_info) .box_err() .with_context(|| ErrWithCause { code: StatusCode::Internal, - msg: format!("fail to create table on shard in cluster, req:{request:?}"), + msg: format!( + "fail to insert table to cluster when creating table on shard, req:{:?}", + request.clone() + ), })?; // Create the table by operator afterwards. @@ -497,14 +556,30 @@ async fn handle_drop_table_on_shard( ctx: HandlerContext, request: DropTableOnShardRequest, ) -> Result<()> { + let updated_table_info = extract_updated_table_info!(request); let shard = ctx .cluster - .drop_table_on_shard(&request) - .await + .get_shard(updated_table_info.shard_info.id) + .with_context(|| ErrNoCause { + code: StatusCode::Internal, + msg: format!( + "shard not found when dropping table on shard, req:{:?}", + request.clone() + ), + })?; + + let mut shard = shard.write().await; + + // FIXME: should insert table from cluster after having dropped table. + shard + .try_remove_table(updated_table_info) .box_err() .with_context(|| ErrWithCause { code: StatusCode::Internal, - msg: format!("fail to drop table on shard in cluster, req:{request:?}"), + msg: format!( + "fail to remove table to cluster when dropping table on shard, req:{:?}", + request.clone() + ), })?; // Drop the table by operator afterwards. @@ -540,13 +615,30 @@ async fn handle_open_table_on_shard( ctx: HandlerContext, request: OpenTableOnShardRequest, ) -> Result<()> { - ctx.cluster - .open_table_on_shard(&request) - .await + let updated_table_info = extract_updated_table_info!(request); + let shard = ctx + .cluster + .get_shard(updated_table_info.shard_info.id) + .with_context(|| ErrNoCause { + code: StatusCode::Internal, + msg: format!( + "shard not found when opening table on shard, req:{:?}", + request.clone() + ), + })?; + + let mut shard = shard.write().await; + + // FIXME: should insert table from cluster after having opened table. + shard + .try_insert_table(updated_table_info) .box_err() .with_context(|| ErrWithCause { code: StatusCode::Internal, - msg: format!("fail to open table on shard in cluster, req:{request:?}"), + msg: format!( + "fail to insert table to cluster when opening table on shard, req:{:?}", + request.clone() + ), })?; // Open the table by operator afterwards. @@ -595,16 +687,33 @@ async fn handle_close_table_on_shard( ctx: HandlerContext, request: CloseTableOnShardRequest, ) -> Result<()> { - let shard = ctx.cluster - .get_shard(shard_id) + let updated_table_info = extract_updated_table_info!(request); + let shard = ctx + .cluster + .get_shard(updated_table_info.shard_info.id) .with_context(|| ErrNoCause { code: StatusCode::Internal, - msg: format!("fail to close table on shard in cluster, req:{:?}", request.clone()), + msg: format!( + "shard not found when closing table on shard, req:{:?}", + request.clone() + ), })?; - let shard = shard.write().await; + let mut shard = shard.write().await; + + // FIXME: should remove table from cluster after having closed table. + shard + .try_remove_table(updated_table_info) + .box_err() + .with_context(|| ErrWithCause { + code: StatusCode::Internal, + msg: format!( + "fail to remove table from cluster when closing table on shard, req:{:?}", + request.clone() + ), + })?; - // Close the table by catalog manager afterwards. + // Close the table by catalog manager afterwards. let catalog_name = &ctx.default_catalog; let table_info = request.table_info.context(ErrNoCause { code: StatusCode::BadRequest, @@ -634,20 +743,6 @@ async fn handle_close_table_on_shard( Ok(()) } -struct UpdateShardInfo { - prev_version: u64, - shard_info: ShardInfo, - table_info: TableInfo, -} - -impl TryFrom for UpdateShardInfo { - type Error = GenericError; - - fn try_from(value: ceresdbproto::meta_event::UpdateShardInfo) -> std::result::Result { - todo!() - } -} - #[async_trait] impl MetaEventService for MetaServiceImpl { async fn open_shard( diff --git a/src/setup.rs b/src/setup.rs index 6d950aa8fd..8a74685761 100644 --- a/src/setup.rs +++ b/src/setup.rs @@ -11,7 +11,7 @@ use analytic_engine::{ }; use catalog::{manager::ManagerRef, schema::OpenOptions, table_operator::TableOperator}; use catalog_impls::{table_based::TableBasedManager, volatile, CatalogManagerImpl}; -use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_tables_cache::ShardSet}; +use cluster::{cluster_impl::ClusterImpl, config::ClusterConfig, shard_set::ShardSet}; use common_util::runtime; use df_operator::registry::FunctionRegistryImpl; use interpreters::table_manipulator::{catalog_based, meta_based}; From c16fbcad933882b5d7dd992274045e9b5032b9a9 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 7 Jul 2023 18:26:12 +0800 Subject: [PATCH 06/15] fix clippy. --- catalog_impls/src/volatile.rs | 4 ++-- cluster/src/cluster_impl.rs | 13 +++---------- cluster/src/lib.rs | 6 +----- cluster/src/shard_set.rs | 2 +- router/src/cluster_based.rs | 5 ----- server/src/grpc/meta_event_service/mod.rs | 4 ++-- 6 files changed, 9 insertions(+), 25 deletions(-) diff --git a/catalog_impls/src/volatile.rs b/catalog_impls/src/volatile.rs index 4ea156c04f..6f752480e4 100644 --- a/catalog_impls/src/volatile.rs +++ b/catalog_impls/src/volatile.rs @@ -308,7 +308,7 @@ impl Schema for SchemaImpl { .get(request.shard_id) .with_context(|| schema::CreateTable { request: request.clone(), - msg: format!("shard not found"), + msg: "shard not found".to_string(), })?; let shard = shard.read().await; @@ -320,7 +320,7 @@ impl Schema for SchemaImpl { }) .with_context(|| schema::CreateTable { request: request.clone(), - msg: format!("table not found in shard"), + msg: "table not found in shard".to_string(), })?; } let request = request.into_engine_create_request(None, self.schema_id); diff --git a/cluster/src/cluster_impl.rs b/cluster/src/cluster_impl.rs index 090ca18e1b..29b6b0fd15 100644 --- a/cluster/src/cluster_impl.rs +++ b/cluster/src/cluster_impl.rs @@ -6,13 +6,7 @@ use std::{ }; use async_trait::async_trait; -use ceresdbproto::{ - meta_event::{ - CloseTableOnShardRequest, CreateTableOnShardRequest, DropTableOnShardRequest, - OpenTableOnShardRequest, UpdateShardInfo, - }, - meta_service::TableInfo as TableInfoPb, -}; + use common_types::table::ShardId; use common_util::{ error::BoxError, @@ -23,7 +17,7 @@ use log::{error, info, warn}; use meta_client::{ types::{ GetNodesRequest, GetTablesOfShardsRequest, RouteTablesRequest, RouteTablesResponse, - ShardInfo, TableInfo, TablesOfShard, + ShardInfo, }, MetaClientRef, }; @@ -38,9 +32,8 @@ use crate::{ shard_lock_manager::{ShardLockManager, ShardLockManagerRef}, shard_set::{ShardRef, ShardSet}, topology::ClusterTopology, - Cluster, ClusterNodesNotFound, ClusterNodesResp, EtcdClientFailureWithCause, Internal, + Cluster, ClusterNodesNotFound, ClusterNodesResp, EtcdClientFailureWithCause, InvalidArguments, MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, - TableNotFound, }; /// ClusterImpl is an implementation of [`Cluster`] based [`MetaClient`]. diff --git a/cluster/src/lib.rs b/cluster/src/lib.rs index f856e39c71..17dba79142 100644 --- a/cluster/src/lib.rs +++ b/cluster/src/lib.rs @@ -13,15 +13,11 @@ use std::sync::Arc; use async_trait::async_trait; -use ceresdbproto::meta_event::{ - CloseTableOnShardRequest, CreateTableOnShardRequest, DropTableOnShardRequest, - OpenTableOnShardRequest, -}; + use common_types::schema::SchemaName; use common_util::{define_result, error::GenericError}; use meta_client::types::{ ClusterNodesRef, RouteTablesRequest, RouteTablesResponse, ShardId, ShardInfo, ShardVersion, - TablesOfShard, }; use shard_lock_manager::ShardLockManagerRef; use snafu::{Backtrace, Snafu}; diff --git a/cluster/src/shard_set.rs b/cluster/src/shard_set.rs index 2a7f5928a3..85eace77ab 100644 --- a/cluster/src/shard_set.rs +++ b/cluster/src/shard_set.rs @@ -5,7 +5,7 @@ use std::{ sync::{Arc, RwLock}, }; -use meta_client::types::{ShardId, ShardInfo, ShardVersion, TableInfo, TablesOfShard}; +use meta_client::types::{ShardId, ShardInfo, TableInfo, TablesOfShard}; use snafu::{ensure, OptionExt}; use crate::{Result, ShardVersionMismatch, TableAlreadyExists, TableNotFound, UpdateFrozenShard}; diff --git a/router/src/cluster_based.rs b/router/src/cluster_based.rs index d72c56c40a..98b71b4e7c 100644 --- a/router/src/cluster_based.rs +++ b/router/src/cluster_based.rs @@ -163,10 +163,6 @@ mod tests { use std::{collections::HashMap, sync::Arc, thread::sleep, time::Duration}; use ceresdbproto::{ - meta_event::{ - CloseTableOnShardRequest, CreateTableOnShardRequest, DropTableOnShardRequest, - OpenTableOnShardRequest, - }, storage::RequestContext, }; use cluster::{ @@ -176,7 +172,6 @@ mod tests { use common_util::config::ReadableDuration; use meta_client::types::{ NodeShard, RouteEntry, RouteTablesResponse, ShardInfo, ShardRole::Leader, TableInfo, - TablesOfShard, }; use super::*; diff --git a/server/src/grpc/meta_event_service/mod.rs b/server/src/grpc/meta_event_service/mod.rs index 9a2a9a4efd..117460a014 100644 --- a/server/src/grpc/meta_event_service/mod.rs +++ b/server/src/grpc/meta_event_service/mod.rs @@ -14,7 +14,7 @@ use catalog::{ table_operator::TableOperator, }; use ceresdbproto::meta_event::{ - self, meta_event_service_server::MetaEventService, ChangeShardRoleRequest, + meta_event_service_server::MetaEventService, ChangeShardRoleRequest, ChangeShardRoleResponse, CloseShardRequest, CloseShardResponse, CloseTableOnShardRequest, CloseTableOnShardResponse, CreateTableOnShardRequest, CreateTableOnShardResponse, DropTableOnShardRequest, DropTableOnShardResponse, MergeShardsRequest, MergeShardsResponse, @@ -24,7 +24,7 @@ use ceresdbproto::meta_event::{ use cluster::{shard_set::UpdatedTableInfo, ClusterRef}; use common_types::{schema::SchemaEncoder, table::ShardId}; use common_util::{ - error::{BoxError, GenericError}, + error::{BoxError}, runtime::Runtime, time::InstantExt, }; From 43a2886494110cee6a0f4c0a3cc0bc147bd9049e Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 9 Jul 2023 23:21:05 +0800 Subject: [PATCH 07/15] refactor `Shard` impl. --- catalog_impls/src/volatile.rs | 8 +- cluster/src/cluster_impl.rs | 22 +-- cluster/src/lib.rs | 1 - cluster/src/shard_set.rs | 162 ++++++++++++++-------- router/src/cluster_based.rs | 4 +- server/src/grpc/meta_event_service/mod.rs | 66 ++++----- 6 files changed, 156 insertions(+), 107 deletions(-) diff --git a/catalog_impls/src/volatile.rs b/catalog_impls/src/volatile.rs index 6f752480e4..2e9ed91437 100644 --- a/catalog_impls/src/volatile.rs +++ b/catalog_impls/src/volatile.rs @@ -311,13 +311,9 @@ impl Schema for SchemaImpl { msg: "shard not found".to_string(), })?; - let shard = shard.read().await; let _ = shard - .tables - .iter() - .find(|table| { - table.schema_name == request.schema_name && table.name == request.table_name - }) + .data + .find_table(&request.schema_name, &request.table_name) .with_context(|| schema::CreateTable { request: request.clone(), msg: "table not found in shard".to_string(), diff --git a/cluster/src/cluster_impl.rs b/cluster/src/cluster_impl.rs index 29b6b0fd15..0af92fc18a 100644 --- a/cluster/src/cluster_impl.rs +++ b/cluster/src/cluster_impl.rs @@ -6,7 +6,6 @@ use std::{ }; use async_trait::async_trait; - use common_types::table::ShardId; use common_util::{ error::BoxError, @@ -30,10 +29,10 @@ use tokio::{ use crate::{ config::ClusterConfig, shard_lock_manager::{ShardLockManager, ShardLockManagerRef}, - shard_set::{ShardRef, ShardSet}, + shard_set::{Shard, ShardRef, ShardSet}, topology::ClusterTopology, - Cluster, ClusterNodesNotFound, ClusterNodesResp, EtcdClientFailureWithCause, - InvalidArguments, MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, + Cluster, ClusterNodesNotFound, ClusterNodesResp, EtcdClientFailureWithCause, InvalidArguments, + MetaClientFailure, OpenShard, OpenShardWithCause, Result, ShardNotFound, }; /// ClusterImpl is an implementation of [`Cluster`] based [`MetaClient`]. @@ -108,8 +107,8 @@ impl ClusterImpl { // TODO: required to acquire the read lock of each shard now, can we optimize // this? for shard in shards { - let shard = shard.read().await; - shard_infos.push(shard.shard_info.clone()); + let shard_info = shard.data.shard_info(); + shard_infos.push(shard_info); } info!("Node heartbeat to meta, shard infos:{:?}", shard_infos); @@ -237,8 +236,9 @@ impl Inner { async fn open_shard(&self, shard_info: &ShardInfo) -> Result { if let Some(shard) = self.shard_set.get(shard_info.id) { - let inner = shard.read().await; - if inner.shard_info.version == shard_info.version { + let shard_data = &shard.data; + let cur_shard_info = shard_data.shard_info(); + if cur_shard_info.version == shard_info.version { info!( "No need to open the exactly same shard again, shard_info:{:?}", shard_info @@ -246,10 +246,10 @@ impl Inner { return Ok(shard.clone()); } ensure!( - inner.shard_info.version < shard_info.version, + cur_shard_info.version < shard_info.version, OpenShard { shard_id: shard_info.id, - msg: format!("open a shard with a smaller version, curr_shard_info:{:?}, new_shard_info:{:?}", inner.shard_info, shard_info), + msg: format!("open a shard with a smaller version, curr_shard_info:{cur_shard_info:?}, new_shard_info:{shard_info:?}"), } ); } @@ -284,7 +284,7 @@ impl Inner { })?; let shard_id = tables_of_shard.shard_info.id; - let shard = Arc::new(tokio::sync::RwLock::new(tables_of_shard.into())); + let shard = Arc::new(Shard::new(tables_of_shard)); self.shard_set.insert(shard_id, shard.clone()); Ok(shard) diff --git a/cluster/src/lib.rs b/cluster/src/lib.rs index 17dba79142..ba303b972b 100644 --- a/cluster/src/lib.rs +++ b/cluster/src/lib.rs @@ -13,7 +13,6 @@ use std::sync::Arc; use async_trait::async_trait; - use common_types::schema::SchemaName; use common_util::{define_result, error::GenericError}; use meta_client::types::{ diff --git a/cluster/src/shard_set.rs b/cluster/src/shard_set.rs index 85eace77ab..73ea9fe6ac 100644 --- a/cluster/src/shard_set.rs +++ b/cluster/src/shard_set.rs @@ -1,9 +1,6 @@ // Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0. -use std::{ - collections::HashMap, - sync::{Arc, RwLock}, -}; +use std::{collections::HashMap, sync::Arc}; use meta_client::types::{ShardId, ShardInfo, TableInfo, TablesOfShard}; use snafu::{ensure, OptionExt}; @@ -14,19 +11,7 @@ use crate::{Result, ShardVersionMismatch, TableAlreadyExists, TableNotFound, Upd /// relationship between them is: one shard -> multiple tables. #[derive(Debug, Default, Clone)] pub struct ShardSet { - inner: Arc>, -} - -#[derive(Debug, Clone)] -pub struct TableWithShards { - pub table_info: TableInfo, - pub shard_infos: Vec, -} - -pub struct UpdatedTableInfo { - pub prev_version: u64, - pub shard_info: ShardInfo, - pub table_info: TableInfo, + inner: Arc>, } impl ShardSet { @@ -51,8 +36,112 @@ impl ShardSet { } } +#[derive(Debug, Default)] +struct ShardSetInner { + // Tables organized by shard. + // TODO: The shard roles should be also taken into considerations. + shards: HashMap, +} + +impl ShardSetInner { + fn all_shard_infos(&self) -> Vec { + self.shards.values().cloned().collect() + } + + fn get(&self, shard_id: ShardId) -> Option { + self.shards.get(&shard_id).cloned() + } + + fn remove(&mut self, shard_id: ShardId) -> Option { + self.shards.remove(&shard_id) + } + + fn insert(&mut self, shard_id: ShardId, shard: ShardRef) { + self.shards.insert(shard_id, shard); + } +} + #[derive(Debug)] pub struct Shard { + pub data: ShardData, + pub serializing_lock: tokio::sync::Mutex<()>, +} + +impl Shard { + pub fn new(tables_of_shard: TablesOfShard) -> Self { + let shard_data_inner = std::sync::RwLock::new(ShardDataInner { + shard_info: tables_of_shard.shard_info, + tables: tables_of_shard.tables, + frozen: false, + }); + let data = ShardData { + inner: shard_data_inner, + }; + + Self { + data, + serializing_lock: tokio::sync::Mutex::new(()), + } + } +} + +pub type ShardRef = Arc; + +#[derive(Debug, Clone)] +pub struct TableWithShards { + pub table_info: TableInfo, + pub shard_infos: Vec, +} + +pub struct UpdatedTableInfo { + pub prev_version: u64, + pub shard_info: ShardInfo, + pub table_info: TableInfo, +} + +#[derive(Debug)] +pub struct ShardData { + inner: std::sync::RwLock, +} + +impl ShardData { + pub fn shard_info(&self) -> ShardInfo { + let inner = self.inner.read().unwrap(); + inner.shard_info.clone() + } + + pub fn find_table(&self, schema_name: &str, table_name: &str) -> Option { + let inner = self.inner.read().unwrap(); + inner + .tables + .iter() + .find(|table| table.schema_name == schema_name && table.name == table_name) + .cloned() + } + + pub fn all_tables(&self) -> Vec { + let inner = self.inner.read().unwrap(); + inner.tables.to_vec() + } + + pub fn freeze(&self) { + let mut inner = self.inner.write().unwrap(); + inner.freeze(); + } + + pub fn try_insert_table(&self, updated_info: UpdatedTableInfo) -> Result<()> { + let mut inner = self.inner.write().unwrap(); + inner.try_insert_table(updated_info) + } + + pub fn try_remove_table(&self, updated_info: UpdatedTableInfo) -> Result<()> { + let mut inner = self.inner.write().unwrap(); + inner.try_remove_table(updated_info) + } +} + +#[derive(Debug)] +pub struct ShardDataInner { /// Shard info pub shard_info: ShardInfo, @@ -63,7 +152,7 @@ pub struct Shard { pub frozen: bool, } -impl Shard { +impl ShardDataInner { pub fn freeze(&mut self) { self.frozen = true; } @@ -142,40 +231,3 @@ impl Shard { Ok(()) } } - -pub type ShardRef = Arc>; - -impl From for Shard { - fn from(value: TablesOfShard) -> Self { - Shard { - shard_info: value.shard_info, - tables: value.tables, - frozen: false, - } - } -} - -#[derive(Debug, Default)] -struct Inner { - // Tables organized by shard. - // TODO: The shard roles should be also taken into considerations. - shards: HashMap, -} - -impl Inner { - fn all_shard_infos(&self) -> Vec { - self.shards.values().cloned().collect() - } - - fn get(&self, shard_id: ShardId) -> Option { - self.shards.get(&shard_id).cloned() - } - - fn remove(&mut self, shard_id: ShardId) -> Option { - self.shards.remove(&shard_id) - } - - fn insert(&mut self, shard_id: ShardId, shard: ShardRef) { - self.shards.insert(shard_id, shard); - } -} diff --git a/router/src/cluster_based.rs b/router/src/cluster_based.rs index 98b71b4e7c..3602c16622 100644 --- a/router/src/cluster_based.rs +++ b/router/src/cluster_based.rs @@ -162,9 +162,7 @@ impl Router for ClusterBasedRouter { mod tests { use std::{collections::HashMap, sync::Arc, thread::sleep, time::Duration}; - use ceresdbproto::{ - storage::RequestContext, - }; + use ceresdbproto::storage::RequestContext; use cluster::{ shard_lock_manager::ShardLockManagerRef, shard_set::ShardRef, Cluster, ClusterNodesResp, }; diff --git a/server/src/grpc/meta_event_service/mod.rs b/server/src/grpc/meta_event_service/mod.rs index 117460a014..9bf17726a6 100644 --- a/server/src/grpc/meta_event_service/mod.rs +++ b/server/src/grpc/meta_event_service/mod.rs @@ -14,20 +14,16 @@ use catalog::{ table_operator::TableOperator, }; use ceresdbproto::meta_event::{ - meta_event_service_server::MetaEventService, ChangeShardRoleRequest, - ChangeShardRoleResponse, CloseShardRequest, CloseShardResponse, CloseTableOnShardRequest, - CloseTableOnShardResponse, CreateTableOnShardRequest, CreateTableOnShardResponse, - DropTableOnShardRequest, DropTableOnShardResponse, MergeShardsRequest, MergeShardsResponse, - OpenShardRequest, OpenShardResponse, OpenTableOnShardRequest, OpenTableOnShardResponse, - SplitShardRequest, SplitShardResponse, + meta_event_service_server::MetaEventService, ChangeShardRoleRequest, ChangeShardRoleResponse, + CloseShardRequest, CloseShardResponse, CloseTableOnShardRequest, CloseTableOnShardResponse, + CreateTableOnShardRequest, CreateTableOnShardResponse, DropTableOnShardRequest, + DropTableOnShardResponse, MergeShardsRequest, MergeShardsResponse, OpenShardRequest, + OpenShardResponse, OpenTableOnShardRequest, OpenTableOnShardResponse, SplitShardRequest, + SplitShardResponse, }; use cluster::{shard_set::UpdatedTableInfo, ClusterRef}; use common_types::{schema::SchemaEncoder, table::ShardId}; -use common_util::{ - error::{BoxError}, - runtime::Runtime, - time::InstantExt, -}; +use common_util::{error::BoxError, runtime::Runtime, time::InstantExt}; use log::{error, info, warn}; use meta_client::types::{ShardInfo, TableInfo}; use paste::paste; @@ -301,18 +297,18 @@ async fn do_open_shard(ctx: HandlerContext, shard_info: ShardInfo) -> Result<()> })?; // Lock the shard in local, and then recover it. - let shard = shard.write().await; + let _guard = shard.serializing_lock.lock().await; let catalog_name = &ctx.default_catalog; - let shard_info = shard.shard_info.clone(); - let table_defs = shard - .tables - .iter() + let shard_info = shard.data.shard_info(); + let tables = shard.data.all_tables(); + let table_defs = tables + .into_iter() .map(|info| TableDef { catalog_name: catalog_name.clone(), - schema_name: info.schema_name.clone(), + schema_name: info.schema_name, id: TableId::from(info.id), - name: info.name.clone(), + name: info.name, }) .collect(); @@ -368,22 +364,22 @@ async fn do_close_shard(ctx: &HandlerContext, shard_id: ShardId) -> Result<()> { msg: format!("shard not found when closing shard, shard_id:{shard_id}",), })?; - let mut shard = shard.write().await; - - shard.freeze(); + // Lock for serializing the write operation. + let _guard = shard.serializing_lock.lock().await; + shard.data.freeze(); info!("Shard is frozen before closed, shard_id:{shard_id}"); let catalog_name = &ctx.default_catalog.clone(); - let shard_info = shard.shard_info.clone(); - let table_defs = shard - .tables - .iter() + let shard_info = shard.data.shard_info(); + let tables = shard.data.all_tables(); + let table_defs = tables + .into_iter() .map(|info| TableDef { catalog_name: catalog_name.clone(), - schema_name: info.schema_name.clone(), + schema_name: info.schema_name, id: TableId::from(info.id), - name: info.name.clone(), + name: info.name, }) .collect(); let close_shard_request = catalog::schema::CloseShardRequest { @@ -463,10 +459,12 @@ async fn handle_create_table_on_shard( ), })?; - let mut shard = shard.write().await; + // Lock for serializing the write operation. + let _guard = shard.serializing_lock.lock().await; // FIXME: should insert table from cluster after having created table. shard + .data .try_insert_table(updated_table_info) .box_err() .with_context(|| ErrWithCause { @@ -568,10 +566,12 @@ async fn handle_drop_table_on_shard( ), })?; - let mut shard = shard.write().await; + // Lock for serializing the write operation. + let _guard = shard.serializing_lock.lock().await; // FIXME: should insert table from cluster after having dropped table. shard + .data .try_remove_table(updated_table_info) .box_err() .with_context(|| ErrWithCause { @@ -627,10 +627,12 @@ async fn handle_open_table_on_shard( ), })?; - let mut shard = shard.write().await; + // Lock for serializing the write operation. + let _guard = shard.serializing_lock.lock().await; // FIXME: should insert table from cluster after having opened table. shard + .data .try_insert_table(updated_table_info) .box_err() .with_context(|| ErrWithCause { @@ -699,10 +701,12 @@ async fn handle_close_table_on_shard( ), })?; - let mut shard = shard.write().await; + // Lock for serializing the write operation. + let _guard = shard.serializing_lock.lock().await; // FIXME: should remove table from cluster after having closed table. shard + .data .try_remove_table(updated_table_info) .box_err() .with_context(|| ErrWithCause { From 6031ba15597d3ed881da74f04f66ec869dfe41aa Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 9 Jul 2023 23:28:28 +0800 Subject: [PATCH 08/15] rename `shard_tables_cache` to `shard_set`. --- catalog_impls/src/volatile.rs | 16 ++++++++-------- cluster/src/cluster_impl.rs | 8 ++++---- server/src/grpc/meta_event_service/mod.rs | 8 ++++---- src/setup.rs | 10 ++++------ 4 files changed, 20 insertions(+), 22 deletions(-) diff --git a/catalog_impls/src/volatile.rs b/catalog_impls/src/volatile.rs index 2e9ed91437..1e195165b4 100644 --- a/catalog_impls/src/volatile.rs +++ b/catalog_impls/src/volatile.rs @@ -32,15 +32,15 @@ use tokio::sync::Mutex; /// ManagerImpl manages multiple volatile catalogs. pub struct ManagerImpl { catalogs: HashMap>, - shard_tables_cache: ShardSet, + shard_set: ShardSet, meta_client: MetaClientRef, } impl ManagerImpl { - pub fn new(shard_tables_cache: ShardSet, meta_client: MetaClientRef) -> Self { + pub fn new(shard_set: ShardSet, meta_client: MetaClientRef) -> Self { let mut manager = ManagerImpl { catalogs: HashMap::new(), - shard_tables_cache, + shard_set, meta_client, }; @@ -87,7 +87,7 @@ impl ManagerImpl { let catalog = Arc::new(CatalogImpl { name: catalog_name.clone(), schemas: RwLock::new(HashMap::new()), - shard_tables_cache: self.shard_tables_cache.clone(), + shard_set: self.shard_set.clone(), meta_client: self.meta_client.clone(), }); @@ -107,7 +107,7 @@ struct CatalogImpl { name: String, /// All the schemas belonging to the catalog. schemas: RwLock>, - shard_tables_cache: ShardSet, + shard_set: ShardSet, meta_client: MetaClientRef, } @@ -156,7 +156,7 @@ impl Catalog for CatalogImpl { self.name.to_string(), name.to_string(), SchemaId::from_u32(schema_id), - self.shard_tables_cache.clone(), + self.shard_set.clone(), )); schemas.insert(name.to_string(), schema); @@ -200,13 +200,13 @@ impl SchemaImpl { catalog_name: String, schema_name: String, schema_id: SchemaId, - shard_tables_cache: ShardSet, + shard_set: ShardSet, ) -> Self { Self { catalog_name, schema_name, schema_id, - shard_set: shard_tables_cache, + shard_set, tables: Default::default(), create_table_mutex: Mutex::new(()), } diff --git a/cluster/src/cluster_impl.rs b/cluster/src/cluster_impl.rs index 0af92fc18a..38015d4ba4 100644 --- a/cluster/src/cluster_impl.rs +++ b/cluster/src/cluster_impl.rs @@ -53,7 +53,7 @@ pub struct ClusterImpl { impl ClusterImpl { pub async fn try_new( node_name: String, - shard_tables_cache: ShardSet, + shard_set: ShardSet, meta_client: MetaClientRef, config: ClusterConfig, runtime: Arc, @@ -62,7 +62,7 @@ impl ClusterImpl { return InvalidArguments { msg: e }.fail(); } - let inner = Arc::new(Inner::new(shard_tables_cache, meta_client)?); + let inner = Arc::new(Inner::new(shard_set, meta_client)?); let connect_options = ConnectOptions::from(&config.etcd_client); let etcd_client = etcd_client::Client::connect(&config.etcd_client.server_addrs, Some(connect_options)) @@ -168,9 +168,9 @@ struct Inner { } impl Inner { - fn new(shard_tables_cache: ShardSet, meta_client: MetaClientRef) -> Result { + fn new(shard_set: ShardSet, meta_client: MetaClientRef) -> Result { Ok(Self { - shard_set: shard_tables_cache, + shard_set, meta_client, topology: Default::default(), }) diff --git a/server/src/grpc/meta_event_service/mod.rs b/server/src/grpc/meta_event_service/mod.rs index 9bf17726a6..e51d77958c 100644 --- a/server/src/grpc/meta_event_service/mod.rs +++ b/server/src/grpc/meta_event_service/mod.rs @@ -462,7 +462,7 @@ async fn handle_create_table_on_shard( // Lock for serializing the write operation. let _guard = shard.serializing_lock.lock().await; - // FIXME: should insert table from cluster after having created table. + // FIXME: maybe should insert table from cluster after having created table. shard .data .try_insert_table(updated_table_info) @@ -569,7 +569,7 @@ async fn handle_drop_table_on_shard( // Lock for serializing the write operation. let _guard = shard.serializing_lock.lock().await; - // FIXME: should insert table from cluster after having dropped table. + // FIXME: maybe should insert table from cluster after having dropped table. shard .data .try_remove_table(updated_table_info) @@ -630,7 +630,7 @@ async fn handle_open_table_on_shard( // Lock for serializing the write operation. let _guard = shard.serializing_lock.lock().await; - // FIXME: should insert table from cluster after having opened table. + // FIXME: maybe should insert table from cluster after having opened table. shard .data .try_insert_table(updated_table_info) @@ -704,7 +704,7 @@ async fn handle_close_table_on_shard( // Lock for serializing the write operation. let _guard = shard.serializing_lock.lock().await; - // FIXME: should remove table from cluster after having closed table. + // FIXME: maybe should remove table from cluster after having closed table. shard .data .try_remove_table(updated_table_info) diff --git a/src/setup.rs b/src/setup.rs index 8a74685761..ac2db96e22 100644 --- a/src/setup.rs +++ b/src/setup.rs @@ -209,11 +209,11 @@ async fn build_with_meta( .await .expect("fail to build meta client"); - let shard_tables_cache = ShardSet::default(); + let shard_set = ShardSet::default(); let cluster = { let cluster_impl = ClusterImpl::try_new( endpoint, - shard_tables_cache.clone(), + shard_set.clone(), meta_client.clone(), cluster_config.clone(), runtimes.meta_runtime.clone(), @@ -238,10 +238,8 @@ async fn build_with_meta( }; let engine_proxy = build_table_engine_proxy(engine_builder).await; - let meta_based_manager_ref = Arc::new(volatile::ManagerImpl::new( - shard_tables_cache, - meta_client.clone(), - )); + let meta_based_manager_ref = + Arc::new(volatile::ManagerImpl::new(shard_set, meta_client.clone())); // Build catalog manager. let catalog_manager = Arc::new(CatalogManagerImpl::new(meta_based_manager_ref)); From 29c480cee36c5a30e694f568b92b679fdcbdb5e8 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 10 Jul 2023 09:35:04 +0800 Subject: [PATCH 09/15] remove outdated todl. --- cluster/src/cluster_impl.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/cluster/src/cluster_impl.rs b/cluster/src/cluster_impl.rs index 38015d4ba4..15f37367bf 100644 --- a/cluster/src/cluster_impl.rs +++ b/cluster/src/cluster_impl.rs @@ -104,8 +104,6 @@ impl ClusterImpl { loop { let shards = inner.shard_set.all_shards(); let mut shard_infos = Vec::with_capacity(shards.len()); - // TODO: required to acquire the read lock of each shard now, can we optimize - // this? for shard in shards { let shard_info = shard.data.shard_info(); shard_infos.push(shard_info); From 6a656bb4d98ca37dfd1bfcdbdd04c178d94567e1 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 10 Jul 2023 10:12:36 +0800 Subject: [PATCH 10/15] add logs to detect dead lock, eliminate some unnecssary converts. --- cluster/src/shard_set.rs | 1 + server/src/grpc/meta_event_service/mod.rs | 133 +++++++++++----------- 2 files changed, 68 insertions(+), 66 deletions(-) diff --git a/cluster/src/shard_set.rs b/cluster/src/shard_set.rs index 73ea9fe6ac..63cad98757 100644 --- a/cluster/src/shard_set.rs +++ b/cluster/src/shard_set.rs @@ -93,6 +93,7 @@ pub struct TableWithShards { pub shard_infos: Vec, } +#[derive(Debug, Clone)] pub struct UpdatedTableInfo { pub prev_version: u64, pub shard_info: ShardInfo, diff --git a/server/src/grpc/meta_event_service/mod.rs b/server/src/grpc/meta_event_service/mod.rs index e51d77958c..1b5e6738e0 100644 --- a/server/src/grpc/meta_event_service/mod.rs +++ b/server/src/grpc/meta_event_service/mod.rs @@ -32,7 +32,6 @@ use query_engine::executor::Executor as QueryExecutor; use snafu::{OptionExt, ResultExt}; use table_engine::{ engine::{TableEngineRef, TableState}, - partition::PartitionInfo, table::TableId, ANALYTIC_ENGINE_TYPE, }; @@ -298,6 +297,7 @@ async fn do_open_shard(ctx: HandlerContext, shard_info: ShardInfo) -> Result<()> // Lock the shard in local, and then recover it. let _guard = shard.serializing_lock.lock().await; + info!("Shard open sequentially begin, shard_id:{}", shard_info.id); let catalog_name = &ctx.default_catalog; let shard_info = shard.data.shard_info(); @@ -328,7 +328,11 @@ async fn do_open_shard(ctx: HandlerContext, shard_info: ShardInfo) -> Result<()> .context(ErrWithCause { code: StatusCode::Internal, msg: "failed to open shard", - }) + })?; + + info!("Shard open sequentially finish, shard_id:{}", shard_info.id); + + Ok(()) } // TODO: maybe we should encapsulate the logic of handling meta event into a @@ -366,6 +370,7 @@ async fn do_close_shard(ctx: &HandlerContext, shard_id: ShardId) -> Result<()> { // Lock for serializing the write operation. let _guard = shard.serializing_lock.lock().await; + info!("Shard close sequentially begin, shard_id:{shard_id}"); shard.data.freeze(); info!("Shard is frozen before closed, shard_id:{shard_id}"); @@ -423,7 +428,11 @@ async fn do_close_shard(ctx: &HandlerContext, shard_id: ShardId) -> Result<()> { ctx.release_shard_lock(shard_id).await.map_err(|e| { error!("Failed to release shard lock, shard_id:{shard_id}, err:{e}"); e - }) + })?; + + info!("Shard close sequentially finish, shard_id:{shard_id}"); + + Ok(()) } async fn handle_close_shard(ctx: HandlerContext, request: CloseShardRequest) -> Result<()> { @@ -448,6 +457,9 @@ async fn handle_create_table_on_shard( request: CreateTableOnShardRequest, ) -> Result<()> { let updated_table_info = extract_updated_table_info!(request); + let shard_info = updated_table_info.shard_info.clone(); + let table_info = updated_table_info.table_info.clone(); + let shard = ctx .cluster .get_shard(updated_table_info.shard_info.id) @@ -461,6 +473,10 @@ async fn handle_create_table_on_shard( // Lock for serializing the write operation. let _guard = shard.serializing_lock.lock().await; + info!( + "Shard create table sequentially begin, shard_id:{}, table_info:{table_info:?}", + shard_info.id + ); // FIXME: maybe should insert table from cluster after having created table. shard @@ -477,21 +493,6 @@ async fn handle_create_table_on_shard( // Create the table by operator afterwards. let catalog_name = &ctx.default_catalog; - let shard_info = request - .update_shard_info - .context(ErrNoCause { - code: StatusCode::BadRequest, - msg: "update shard info is missing in the CreateTableOnShardRequest", - })? - .curr_shard_info - .context(ErrNoCause { - code: StatusCode::BadRequest, - msg: "current shard info is missing ine CreateTableOnShardRequest", - })?; - let table_info = request.table_info.context(ErrNoCause { - code: StatusCode::BadRequest, - msg: "table info is missing in the CreateTableOnShardRequest", - })?; // Get information for partition table creating. let table_schema = SchemaEncoder::default() @@ -505,24 +506,16 @@ async fn handle_create_table_on_shard( ), })?; - let (table_engine, partition_info) = match table_info.partition_info { - Some(v) => { - let partition_info = Some(PartitionInfo::try_from(v.clone()).box_err().with_context( - || ErrWithCause { - code: StatusCode::BadRequest, - msg: format!("fail to parse partition info, partition_info:{v:?}"), - }, - )?); - (ctx.partition_table_engine.clone(), partition_info) - } + let (table_engine, partition_info) = match table_info.partition_info.clone() { + Some(v) => (ctx.partition_table_engine.clone(), Some(v)), None => (ctx.table_engine.clone(), None), }; // Build create table request and options. let create_table_request = CreateTableRequest { catalog_name: catalog_name.clone(), - schema_name: table_info.schema_name, - table_name: table_info.name, + schema_name: table_info.schema_name.clone(), + table_name: table_info.name.clone(), table_id: Some(TableId::new(table_info.id)), table_schema, engine: request.engine, @@ -547,6 +540,11 @@ async fn handle_create_table_on_shard( msg: format!("fail to create table with request:{create_table_request:?}"), })?; + info!( + "Shard create table sequentially begin, shard_id:{}, table_info:{table_info:?}", + shard_info.id + ); + Ok(()) } @@ -555,6 +553,9 @@ async fn handle_drop_table_on_shard( request: DropTableOnShardRequest, ) -> Result<()> { let updated_table_info = extract_updated_table_info!(request); + let shard_info = updated_table_info.shard_info.clone(); + let table_info = updated_table_info.table_info.clone(); + let shard = ctx .cluster .get_shard(updated_table_info.shard_info.id) @@ -568,6 +569,10 @@ async fn handle_drop_table_on_shard( // Lock for serializing the write operation. let _guard = shard.serializing_lock.lock().await; + info!( + "Shard drop table sequentially begin, shard_id:{}, table_info:{table_info:?}", + shard_info.id + ); // FIXME: maybe should insert table from cluster after having dropped table. shard @@ -584,14 +589,10 @@ async fn handle_drop_table_on_shard( // Drop the table by operator afterwards. let catalog_name = ctx.default_catalog.clone(); - let table_info = request.table_info.context(ErrNoCause { - code: StatusCode::BadRequest, - msg: "table info is missing in the DropTableOnShardRequest", - })?; let drop_table_request = DropTableRequest { catalog_name, - schema_name: table_info.schema_name, - table_name: table_info.name, + schema_name: table_info.schema_name.clone(), + table_name: table_info.name.clone(), // FIXME: the engine type should not use the default one. engine: ANALYTIC_ENGINE_TYPE.to_string(), }; @@ -608,6 +609,11 @@ async fn handle_drop_table_on_shard( msg: format!("fail to drop table with request:{drop_table_request:?}"), })?; + info!( + "Shard drop table sequentially finish, shard_id:{}, table_info:{table_info:?}", + shard_info.id + ); + Ok(()) } @@ -616,19 +622,23 @@ async fn handle_open_table_on_shard( request: OpenTableOnShardRequest, ) -> Result<()> { let updated_table_info = extract_updated_table_info!(request); + let shard_info = updated_table_info.shard_info.clone(); + let table_info = updated_table_info.table_info.clone(); + let shard = ctx .cluster .get_shard(updated_table_info.shard_info.id) .with_context(|| ErrNoCause { code: StatusCode::Internal, - msg: format!( - "shard not found when opening table on shard, req:{:?}", - request.clone() - ), + msg: format!("shard not found when opening table on shard, request:{request:?}",), })?; // Lock for serializing the write operation. let _guard = shard.serializing_lock.lock().await; + info!( + "Shard open table sequentially begin, shard_id:{}, table_info:{table_info:?}", + shard_info.id + ); // FIXME: maybe should insert table from cluster after having opened table. shard @@ -638,32 +648,16 @@ async fn handle_open_table_on_shard( .with_context(|| ErrWithCause { code: StatusCode::Internal, msg: format!( - "fail to insert table to cluster when opening table on shard, req:{:?}", - request.clone() + "fail to insert table to cluster when opening table on shard, request:{request:?}", ), })?; // Open the table by operator afterwards. let catalog_name = ctx.default_catalog.clone(); - let shard_info = request - .update_shard_info - .context(ErrNoCause { - code: StatusCode::BadRequest, - msg: "update shard info is missing in the OpenTableOnShardRequest", - })? - .curr_shard_info - .context(ErrNoCause { - code: StatusCode::BadRequest, - msg: "current shard info is missing ine OpenTableOnShardRequest", - })?; - let table_info = request.table_info.context(ErrNoCause { - code: StatusCode::BadRequest, - msg: "table info is missing in the OpenTableOnShardRequest", - })?; let open_table_request = OpenTableRequest { catalog_name, - schema_name: table_info.schema_name, - table_name: table_info.name, + schema_name: table_info.schema_name.clone(), + table_name: table_info.name.clone(), // FIXME: the engine type should not use the default one. engine: ANALYTIC_ENGINE_TYPE.to_string(), shard_id: shard_info.id, @@ -682,6 +676,11 @@ async fn handle_open_table_on_shard( msg: format!("fail to open table with request:{open_table_request:?}"), })?; + info!( + "Shard open table sequentially finish, shard_id:{}, table_info:{table_info:?}", + shard_info.id + ); + Ok(()) } @@ -690,9 +689,12 @@ async fn handle_close_table_on_shard( request: CloseTableOnShardRequest, ) -> Result<()> { let updated_table_info = extract_updated_table_info!(request); + let shard_id = updated_table_info.shard_info.id; + let table_info = updated_table_info.table_info.clone(); + let shard = ctx .cluster - .get_shard(updated_table_info.shard_info.id) + .get_shard(shard_id) .with_context(|| ErrNoCause { code: StatusCode::Internal, msg: format!( @@ -703,6 +705,7 @@ async fn handle_close_table_on_shard( // Lock for serializing the write operation. let _guard = shard.serializing_lock.lock().await; + info!("Shard close table sequentially begin, shard_id:{shard_id}, table_info:{table_info:?}"); // FIXME: maybe should remove table from cluster after having closed table. shard @@ -719,14 +722,10 @@ async fn handle_close_table_on_shard( // Close the table by catalog manager afterwards. let catalog_name = &ctx.default_catalog; - let table_info = request.table_info.context(ErrNoCause { - code: StatusCode::BadRequest, - msg: "table info is missing in the CloseTableOnShardRequest", - })?; let close_table_request = CloseTableRequest { catalog_name: catalog_name.clone(), - schema_name: table_info.schema_name, - table_name: table_info.name, + schema_name: table_info.schema_name.clone(), + table_name: table_info.name.clone(), table_id: TableId::new(table_info.id), // FIXME: the engine type should not use the default one. engine: ANALYTIC_ENGINE_TYPE.to_string(), @@ -744,6 +743,8 @@ async fn handle_close_table_on_shard( msg: format!("fail to close table with request:{close_table_request:?}"), })?; + info!("Shard close table sequentially finish, shard_id:{shard_id}, table_info:{table_info:?}"); + Ok(()) } From 98f4222f273c036c5c1b2a6e4da143a4dc25d7b3 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 10 Jul 2023 20:28:21 +0800 Subject: [PATCH 11/15] move shard operation logic from meta event service to `Shard`. --- Cargo.lock | 2 + catalog_impls/src/volatile.rs | 2 +- cluster/Cargo.toml | 2 + cluster/src/cluster_impl.rs | 11 +- cluster/src/lib.rs | 46 +- .../src}/shard_operation.rs | 0 cluster/src/shard_operator.rs | 406 ++++++++++++++++++ cluster/src/shard_set.rs | 158 ++++--- server/src/grpc/meta_event_service/mod.rs | 346 +++------------ 9 files changed, 598 insertions(+), 375 deletions(-) rename {server/src/grpc/meta_event_service => cluster/src}/shard_operation.rs (100%) create mode 100644 cluster/src/shard_operator.rs diff --git a/Cargo.lock b/Cargo.lock index 7a9d975e0f..c7138a39a1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1239,6 +1239,7 @@ name = "cluster" version = "1.2.2" dependencies = [ "async-trait", + "catalog", "ceresdbproto", "common_types", "common_util", @@ -1251,6 +1252,7 @@ dependencies = [ "snafu 0.6.10", "table_engine", "tokio", + "wal", ] [[package]] diff --git a/catalog_impls/src/volatile.rs b/catalog_impls/src/volatile.rs index 1e195165b4..fda87ae401 100644 --- a/catalog_impls/src/volatile.rs +++ b/catalog_impls/src/volatile.rs @@ -311,8 +311,8 @@ impl Schema for SchemaImpl { msg: "shard not found".to_string(), })?; + // TODO: seems unnecessary? let _ = shard - .data .find_table(&request.schema_name, &request.table_name) .with_context(|| schema::CreateTable { request: request.clone(), diff --git a/cluster/Cargo.toml b/cluster/Cargo.toml index 2acbf9ab5d..6b40f42382 100644 --- a/cluster/Cargo.toml +++ b/cluster/Cargo.toml @@ -12,6 +12,7 @@ workspace = true [dependencies] async-trait = { workspace = true } +catalog = { workspace = true } ceresdbproto = { workspace = true } common_types = { workspace = true } common_util = { workspace = true } @@ -24,3 +25,4 @@ serde_json = { workspace = true } snafu = { workspace = true } table_engine = { workspace = true } tokio = { workspace = true } +wal = { workspace = true } diff --git a/cluster/src/cluster_impl.rs b/cluster/src/cluster_impl.rs index 15f37367bf..ce90b3b16c 100644 --- a/cluster/src/cluster_impl.rs +++ b/cluster/src/cluster_impl.rs @@ -105,7 +105,7 @@ impl ClusterImpl { let shards = inner.shard_set.all_shards(); let mut shard_infos = Vec::with_capacity(shards.len()); for shard in shards { - let shard_info = shard.data.shard_info(); + let shard_info = shard.shard_info(); shard_infos.push(shard_info); } info!("Node heartbeat to meta, shard infos:{:?}", shard_infos); @@ -234,14 +234,13 @@ impl Inner { async fn open_shard(&self, shard_info: &ShardInfo) -> Result { if let Some(shard) = self.shard_set.get(shard_info.id) { - let shard_data = &shard.data; - let cur_shard_info = shard_data.shard_info(); + let cur_shard_info = shard.shard_info(); if cur_shard_info.version == shard_info.version { info!( "No need to open the exactly same shard again, shard_info:{:?}", shard_info ); - return Ok(shard.clone()); + return Ok(shard); } ensure!( cur_shard_info.version < shard_info.version, @@ -261,8 +260,8 @@ impl Inner { .get_tables_of_shards(req) .await .box_err() - .context(OpenShardWithCause { - shard_id: shard_info.id, + .with_context(|| OpenShardWithCause { + msg: format!("shard_info:{shard_info:?}"), })?; ensure!( diff --git a/cluster/src/lib.rs b/cluster/src/lib.rs index ba303b972b..cf3dfd1dea 100644 --- a/cluster/src/lib.rs +++ b/cluster/src/lib.rs @@ -26,6 +26,8 @@ use crate::shard_set::ShardRef; pub mod cluster_impl; pub mod config; pub mod shard_lock_manager; +pub mod shard_operation; +pub mod shard_operator; pub mod shard_set; #[allow(dead_code)] pub mod topology; @@ -64,17 +66,41 @@ pub enum Error { backtrace: Backtrace, }, - #[snafu(display("Fail to open shard, shard_id:{shard_id}, source:{source}."))] - OpenShardWithCause { - shard_id: ShardId, - source: GenericError, - }, + #[snafu(display("Fail to open shard, msg:{msg}, source:{source}."))] + OpenShardWithCause { msg: String, source: GenericError }, - #[snafu(display("Fail to close shard, shard_id:{shard_id}, source:{source}."))] - CloseShardWithCause { - shard_id: ShardId, - source: GenericError, - }, + #[snafu(display("Fail to open shard, msg:{msg}.\nBacktrace:\n{backtrace}"))] + OpenShardNoCause { msg: String, backtrace: Backtrace }, + + #[snafu(display("Fail to close shard, msg:{msg}, source:{source}."))] + CloseShardWithCause { msg: String, source: GenericError }, + + #[snafu(display("Fail to close shard, msg:{msg}.\nBacktrace:\n{backtrace}"))] + CloseShardNoCause { msg: String, backtrace: Backtrace }, + + #[snafu(display("Fail to create table on shard, msg:{msg}, source:{source}."))] + CreateTableWithCause { msg: String, source: GenericError }, + + #[snafu(display("Fail to create table on shard, msg:{msg}.\nBacktrace:\n{backtrace}"))] + CreateTableNoCause { msg: String, backtrace: Backtrace }, + + #[snafu(display("Fail to drop table on shard, msg:{msg}, source:{source}."))] + DropTableWithCause { msg: String, source: GenericError }, + + #[snafu(display("Fail to drop table on shard, msg:{msg}.\nBacktrace:\n{backtrace}"))] + DropTableNoCause { msg: String, backtrace: Backtrace }, + + #[snafu(display("Fail to open table on shard, msg:{msg}, source:{source}."))] + OpenTableWithCause { msg: String, source: GenericError }, + + #[snafu(display("Fail to open table on shard, msg:{msg}.\nBacktrace:\n{backtrace}"))] + OpenTableNoCause { msg: String, backtrace: Backtrace }, + + #[snafu(display("Fail to close table on shard, msg:{msg}, source:{source}."))] + CloseTableWithCause { msg: String, source: GenericError }, + + #[snafu(display("Fail to close table on shard, msg:{msg}.\nBacktrace:\n{backtrace}"))] + CloseTableNoCause { msg: String, backtrace: Backtrace }, #[snafu(display("Shard not found, msg:{msg}.\nBacktrace:\n{backtrace}"))] ShardNotFound { msg: String, backtrace: Backtrace }, diff --git a/server/src/grpc/meta_event_service/shard_operation.rs b/cluster/src/shard_operation.rs similarity index 100% rename from server/src/grpc/meta_event_service/shard_operation.rs rename to cluster/src/shard_operation.rs diff --git a/cluster/src/shard_operator.rs b/cluster/src/shard_operator.rs new file mode 100644 index 0000000000..4d039d39f2 --- /dev/null +++ b/cluster/src/shard_operator.rs @@ -0,0 +1,406 @@ +// Copyright 2023 CeresDB Project Authors. Licensed under Apache-2.0. + +use std::collections::HashMap; + +use catalog::{ + schema::{ + CloseOptions, CloseTableRequest, CreateOptions, CreateTableRequest, DropOptions, + DropTableRequest, OpenOptions, OpenTableRequest, TableDef, + }, + table_operator::TableOperator, +}; +use common_util::error::BoxError; +use log::info; +use snafu::ResultExt; +use table_engine::{ + engine::{TableEngineRef, TableState}, + table::TableId, + ANALYTIC_ENGINE_TYPE, +}; + +use crate::{ + shard_operation::WalRegionCloserRef, + shard_set::{ShardDataRef, UpdatedTableInfo}, + CloseShardWithCause, CloseTableWithCause, CreateTableWithCause, DropTableWithCause, + OpenShardWithCause, OpenTableWithCause, Result, +}; + +pub struct OpenContext { + pub catalog: String, + pub table_engine: TableEngineRef, + pub table_operator: TableOperator, + pub engine: String, +} + +impl std::fmt::Debug for OpenContext { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("OpenContext") + .field("catalog", &self.catalog) + .field("engine", &self.engine) + .finish() + } +} + +pub struct CloseContext { + pub catalog: String, + pub table_engine: TableEngineRef, + pub table_operator: TableOperator, + pub wal_region_closer: WalRegionCloserRef, + pub engine: String, +} + +impl std::fmt::Debug for CloseContext { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("OpenContext") + .field("catalog", &self.catalog) + .field("engine", &self.engine) + .finish() + } +} + +pub struct CreateTableContext { + pub catalog: String, + pub table_engine: TableEngineRef, + pub table_operator: TableOperator, + pub partition_table_engine: TableEngineRef, + pub updated_table_info: UpdatedTableInfo, + pub table_schema: common_types::schema::Schema, + pub options: HashMap, + pub create_if_not_exist: bool, + pub engine: String, +} + +impl std::fmt::Debug for CreateTableContext { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("CreateTableContext") + .field("catalog", &self.catalog) + .field("updated_table_info", &self.updated_table_info) + .field("table_schema", &self.table_schema) + .field("options", &self.options) + .field("engine", &self.engine) + .field("create_if_not_exist", &self.create_if_not_exist) + .finish() + } +} + +pub struct AlterContext { + pub catalog: String, + pub table_engine: TableEngineRef, + pub table_operator: TableOperator, + pub updated_table_info: UpdatedTableInfo, + pub engine: String, +} + +impl std::fmt::Debug for AlterContext { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("AlterContext") + .field("catalog", &self.catalog) + .field("updated_table_info", &self.updated_table_info) + .field("engine", &self.engine) + .finish() + } +} + +pub type DropTableContext = AlterContext; +pub type OpenTableContext = AlterContext; +pub type CloseTableContext = AlterContext; + +pub struct ShardOperator { + pub data: ShardDataRef, +} + +impl ShardOperator { + pub async fn open(&self, ctx: OpenContext) -> Result<()> { + let (shard_info, tables) = { + let data = self.data.read().unwrap(); + let shard_info = data.shard_info.clone(); + let tables = data.tables.clone(); + + (shard_info, tables) + }; + + // Lock the shard in local, and then recover it. + info!("ShardOperator open sequentially begin, shard_id:{shard_info:?}"); + + let table_defs = tables + .into_iter() + .map(|info| TableDef { + catalog_name: ctx.catalog.clone(), + schema_name: info.schema_name, + id: TableId::from(info.id), + name: info.name, + }) + .collect(); + + let open_shard_request = catalog::schema::OpenShardRequest { + shard_id: shard_info.id, + table_defs, + engine: ctx.engine, + }; + let opts = OpenOptions { + table_engine: ctx.table_engine.clone(), + }; + + ctx.table_operator + .open_shard(open_shard_request, opts) + .await + .box_err() + .with_context(|| OpenShardWithCause { + msg: format!("shard_info:{shard_info:?}"), + })?; + + info!("ShardOperator open sequentially finish, shard_id:{shard_info:?}"); + + Ok(()) + } + + pub async fn close(&self, ctx: CloseContext) -> Result<()> { + let (shard_info, tables) = { + let data = self.data.read().unwrap(); + let shard_info = data.shard_info.clone(); + let tables = data.tables.clone(); + + (shard_info, tables) + }; + info!("ShardOperator close sequentially begin, shard_info:{shard_info:?}"); + + { + let mut data = self.data.write().unwrap(); + data.freeze(); + info!("Shard is frozen before closed, shard_id:{}", shard_info.id); + } + + let table_defs = tables + .into_iter() + .map(|info| TableDef { + catalog_name: ctx.catalog.clone(), + schema_name: info.schema_name, + id: TableId::from(info.id), + name: info.name, + }) + .collect(); + let close_shard_request = catalog::schema::CloseShardRequest { + shard_id: shard_info.id, + table_defs, + engine: ctx.engine, + }; + let opts = CloseOptions { + table_engine: ctx.table_engine, + }; + + ctx.table_operator + .close_shard(close_shard_request, opts) + .await + .box_err() + .with_context(|| CloseShardWithCause { + msg: format!("shard_info:{shard_info:?}"), + })?; + + // Try to close wal region + ctx.wal_region_closer + .close_region(shard_info.id) + .await + .with_context(|| CloseShardWithCause { + msg: format!("shard_info:{shard_info:?}"), + })?; + + info!("ShardOperator close sequentially finish, shard_info:{shard_info:?}"); + + Ok(()) + } + + pub async fn create_table(&self, ctx: CreateTableContext) -> Result<()> { + let shard_info = ctx.updated_table_info.shard_info.clone(); + let table_info = ctx.updated_table_info.table_info.clone(); + + info!( + "ShardOperator create table sequentially begin, shard_info:{shard_info:?}, table_info:{table_info:?}", + ); + + // FIXME: maybe should insert table from cluster after having created table. + { + let mut data = self.data.write().unwrap(); + data.try_insert_table(ctx.updated_table_info) + .box_err() + .with_context(|| CreateTableWithCause { + msg: format!("shard_info:{shard_info:?}, table_info:{table_info:?}"), + })?; + } + + // Create the table by operator afterwards. + let catalog_name = ctx.catalog; + + let (table_engine, partition_info) = match table_info.partition_info.clone() { + Some(v) => (ctx.partition_table_engine.clone(), Some(v)), + None => (ctx.table_engine.clone(), None), + }; + + // Build create table request and options. + let create_table_request = CreateTableRequest { + catalog_name: catalog_name.clone(), + schema_name: table_info.schema_name.clone(), + table_name: table_info.name.clone(), + table_id: Some(TableId::new(table_info.id)), + table_schema: ctx.table_schema, + engine: ctx.engine, + options: ctx.options, + state: TableState::Stable, + shard_id: shard_info.id, + partition_info, + }; + + let create_opts = CreateOptions { + table_engine, + create_if_not_exists: ctx.create_if_not_exist, + }; + + let _ = ctx + .table_operator + .create_table_on_shard(create_table_request.clone(), create_opts) + .await + .box_err() + .with_context(|| CreateTableWithCause { + msg: format!("shard_info:{shard_info:?}, table_info:{table_info:?}"), + })?; + + info!( + "ShardOperator create table sequentially finish, shard_info:{shard_info:?}, table_info:{table_info:?}", + ); + + Ok(()) + } + + pub async fn drop_table(&self, ctx: DropTableContext) -> Result<()> { + let shard_info = ctx.updated_table_info.shard_info.clone(); + let table_info = ctx.updated_table_info.table_info.clone(); + + info!( + "ShardOperator drop table sequentially begin, shard_info:{shard_info:?}, table_info:{table_info:?}", + ); + + // FIXME: maybe should insert table from cluster after having dropped table. + { + let mut data = self.data.write().unwrap(); + data.try_remove_table(ctx.updated_table_info) + .box_err() + .with_context(|| DropTableWithCause { + msg: format!("shard_info:{shard_info:?}, table_info:{table_info:?}"), + })?; + } + + // Drop the table by operator afterwards. + let drop_table_request = DropTableRequest { + catalog_name: ctx.catalog, + schema_name: table_info.schema_name.clone(), + table_name: table_info.name.clone(), + // FIXME: the engine type should not use the default one. + engine: ANALYTIC_ENGINE_TYPE.to_string(), + }; + let drop_opts = DropOptions { + table_engine: ctx.table_engine, + }; + + ctx.table_operator + .drop_table_on_shard(drop_table_request.clone(), drop_opts) + .await + .box_err() + .with_context(|| DropTableWithCause { + msg: format!("shard_info:{shard_info:?}, table_info:{table_info:?}"), + })?; + + info!( + "ShardOperator drop table sequentially finish, shard_info:{shard_info:?}, table_info:{table_info:?}", + ); + + Ok(()) + } + + pub async fn open_table(&self, ctx: OpenTableContext) -> Result<()> { + let shard_info = ctx.updated_table_info.shard_info.clone(); + let table_info = ctx.updated_table_info.table_info.clone(); + + info!( + "ShardOperator open table sequentially begin, shard_info:{shard_info:?}, table_info:{table_info:?}", + ); + + // FIXME: maybe should insert table from cluster after having opened table. + { + let mut data = self.data.write().unwrap(); + data.try_insert_table(ctx.updated_table_info) + .box_err() + .with_context(|| OpenTableWithCause { + msg: format!("shard_info:{shard_info:?}, table_info:{table_info:?}"), + })?; + } + + // Open the table by operator afterwards. + let open_table_request = OpenTableRequest { + catalog_name: ctx.catalog, + schema_name: table_info.schema_name.clone(), + table_name: table_info.name.clone(), + // FIXME: the engine type should not use the default one. + engine: ctx.engine, + shard_id: shard_info.id, + table_id: TableId::new(table_info.id), + }; + let open_opts = OpenOptions { + table_engine: ctx.table_engine, + }; + + ctx.table_operator + .open_table_on_shard(open_table_request.clone(), open_opts) + .await + .box_err() + .with_context(|| OpenTableWithCause { + msg: format!("shard_info:{shard_info:?}, table_info:{table_info:?}"), + })?; + + info!( + "ShardOperator open table sequentially finish, shard_info:{shard_info:?}, table_info:{table_info:?}", + ); + + Ok(()) + } + + pub async fn close_table(&self, ctx: CloseTableContext) -> Result<()> { + let shard_info = ctx.updated_table_info.shard_info.clone(); + let table_info = ctx.updated_table_info.table_info.clone(); + + info!("ShardOperator close table sequentially begin, shard_info:{shard_info:?}, table_info:{table_info:?}"); + + // FIXME: maybe should remove table from cluster after having closed table. + { + let mut data = self.data.write().unwrap(); + data.try_remove_table(ctx.updated_table_info) + .box_err() + .with_context(|| CloseTableWithCause { + msg: format!("shard_info:{shard_info:?}, table_info:{table_info:?}"), + })?; + } + + // Close the table by catalog manager afterwards. + let close_table_request = CloseTableRequest { + catalog_name: ctx.catalog, + schema_name: table_info.schema_name.clone(), + table_name: table_info.name.clone(), + table_id: TableId::new(table_info.id), + // FIXME: the engine type should not use the default one. + engine: ctx.engine, + }; + let close_opts = CloseOptions { + table_engine: ctx.table_engine, + }; + + ctx.table_operator + .close_table_on_shard(close_table_request.clone(), close_opts) + .await + .box_err() + .with_context(|| CloseTableWithCause { + msg: format!("shard_info:{shard_info:?}, table_info:{table_info:?}"), + })?; + + info!("ShardOperator close table sequentially finish, shard_info:{shard_info:?}, table_info:{table_info:?}"); + + Ok(()) + } +} diff --git a/cluster/src/shard_set.rs b/cluster/src/shard_set.rs index 63cad98757..a537964ffe 100644 --- a/cluster/src/shard_set.rs +++ b/cluster/src/shard_set.rs @@ -5,83 +5,109 @@ use std::{collections::HashMap, sync::Arc}; use meta_client::types::{ShardId, ShardInfo, TableInfo, TablesOfShard}; use snafu::{ensure, OptionExt}; -use crate::{Result, ShardVersionMismatch, TableAlreadyExists, TableNotFound, UpdateFrozenShard}; +use crate::{ + shard_operator::{ + CloseContext, CloseTableContext, CreateTableContext, DropTableContext, OpenContext, + OpenTableContext, ShardOperator, + }, + Result, ShardVersionMismatch, TableAlreadyExists, TableNotFound, UpdateFrozenShard, +}; /// [ShardTablesCache] caches the information about tables and shards, and the /// relationship between them is: one shard -> multiple tables. #[derive(Debug, Default, Clone)] pub struct ShardSet { - inner: Arc>, + inner: Arc>>, } impl ShardSet { // Fetch all the shard infos. pub fn all_shards(&self) -> Vec { - self.inner.read().unwrap().all_shard_infos() + let inner = self.inner.read().unwrap(); + inner.values().cloned().collect() } // Get the shard by its id. pub fn get(&self, shard_id: ShardId) -> Option { - self.inner.read().unwrap().get(shard_id) + let inner = self.inner.read().unwrap(); + inner.get(&shard_id).cloned() } /// Remove the shard. pub fn remove(&self, shard_id: ShardId) -> Option { - self.inner.write().unwrap().remove(shard_id) + let mut inner = self.inner.write().unwrap(); + inner.remove(&shard_id) } /// Insert the tables of one shard. pub fn insert(&self, shard_id: ShardId, shard: ShardRef) { - self.inner.write().unwrap().insert(shard_id, shard) + let mut inner = self.inner.write().unwrap(); + inner.insert(shard_id, shard); } } -#[derive(Debug, Default)] -struct ShardSetInner { - // Tables organized by shard. - // TODO: The shard roles should be also taken into considerations. - shards: HashMap, +pub struct Shard { + pub data: ShardDataRef, + pub operator: tokio::sync::Mutex, } -impl ShardSetInner { - fn all_shard_infos(&self) -> Vec { - self.shards.values().cloned().collect() +impl std::fmt::Debug for Shard { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("Shard").field("data", &self.data).finish() } +} + +impl Shard { + pub fn new(tables_of_shard: TablesOfShard) -> Self { + let data = Arc::new(std::sync::RwLock::new(ShardData { + shard_info: tables_of_shard.shard_info, + tables: tables_of_shard.tables, + frozen: false, + })); - fn get(&self, shard_id: ShardId) -> Option { - self.shards.get(&shard_id).cloned() + let operator = tokio::sync::Mutex::new(ShardOperator { data: data.clone() }); + + Self { data, operator } } - fn remove(&mut self, shard_id: ShardId) -> Option { - self.shards.remove(&shard_id) + pub fn shard_info(&self) -> ShardInfo { + let data = self.data.read().unwrap(); + data.shard_info.clone() } - fn insert(&mut self, shard_id: ShardId, shard: ShardRef) { - self.shards.insert(shard_id, shard); + pub fn find_table(&self, schema_name: &str, table_name: &str) -> Option { + let data = self.data.read().unwrap(); + data.find_table(schema_name, table_name) } -} -#[derive(Debug)] -pub struct Shard { - pub data: ShardData, - pub serializing_lock: tokio::sync::Mutex<()>, -} + pub async fn open(&self, ctx: OpenContext) -> Result<()> { + let operator = self.operator.lock().await; + operator.open(ctx).await + } -impl Shard { - pub fn new(tables_of_shard: TablesOfShard) -> Self { - let shard_data_inner = std::sync::RwLock::new(ShardDataInner { - shard_info: tables_of_shard.shard_info, - tables: tables_of_shard.tables, - frozen: false, - }); - let data = ShardData { - inner: shard_data_inner, - }; + pub async fn close(&self, ctx: CloseContext) -> Result<()> { + let operator = self.operator.lock().await; + operator.close(ctx).await + } - Self { - data, - serializing_lock: tokio::sync::Mutex::new(()), - } + pub async fn create_table(&self, ctx: CreateTableContext) -> Result<()> { + let operator = self.operator.lock().await; + operator.create_table(ctx).await + } + + pub async fn drop_table(&self, ctx: DropTableContext) -> Result<()> { + let operator = self.operator.lock().await; + operator.drop_table(ctx).await + } + + pub async fn open_table(&self, ctx: OpenTableContext) -> Result<()> { + let operator = self.operator.lock().await; + operator.open_table(ctx).await + } + + pub async fn close_table(&self, ctx: CloseTableContext) -> Result<()> { + let operator = self.operator.lock().await; + operator.close_table(ctx).await } } @@ -102,47 +128,6 @@ pub struct UpdatedTableInfo { #[derive(Debug)] pub struct ShardData { - inner: std::sync::RwLock, -} - -impl ShardData { - pub fn shard_info(&self) -> ShardInfo { - let inner = self.inner.read().unwrap(); - inner.shard_info.clone() - } - - pub fn find_table(&self, schema_name: &str, table_name: &str) -> Option { - let inner = self.inner.read().unwrap(); - inner - .tables - .iter() - .find(|table| table.schema_name == schema_name && table.name == table_name) - .cloned() - } - - pub fn all_tables(&self) -> Vec { - let inner = self.inner.read().unwrap(); - inner.tables.to_vec() - } - - pub fn freeze(&self) { - let mut inner = self.inner.write().unwrap(); - inner.freeze(); - } - - pub fn try_insert_table(&self, updated_info: UpdatedTableInfo) -> Result<()> { - let mut inner = self.inner.write().unwrap(); - inner.try_insert_table(updated_info) - } - - pub fn try_remove_table(&self, updated_info: UpdatedTableInfo) -> Result<()> { - let mut inner = self.inner.write().unwrap(); - inner.try_remove_table(updated_info) - } -} - -#[derive(Debug)] -pub struct ShardDataInner { /// Shard info pub shard_info: ShardInfo, @@ -153,7 +138,14 @@ pub struct ShardDataInner { pub frozen: bool, } -impl ShardDataInner { +impl ShardData { + pub fn find_table(&self, schema_name: &str, table_name: &str) -> Option { + self.tables + .iter() + .find(|table| table.schema_name == schema_name && table.name == table_name) + .cloned() + } + pub fn freeze(&mut self) { self.frozen = true; } @@ -232,3 +224,5 @@ impl ShardDataInner { Ok(()) } } + +pub type ShardDataRef = Arc>; diff --git a/server/src/grpc/meta_event_service/mod.rs b/server/src/grpc/meta_event_service/mod.rs index 1b5e6738e0..42d575c591 100644 --- a/server/src/grpc/meta_event_service/mod.rs +++ b/server/src/grpc/meta_event_service/mod.rs @@ -6,13 +6,7 @@ use std::{sync::Arc, time::Instant}; use analytic_engine::setup::OpenedWals; use async_trait::async_trait; -use catalog::{ - schema::{ - CloseOptions, CloseTableRequest, CreateOptions, CreateTableRequest, DropOptions, - DropTableRequest, OpenOptions, OpenTableRequest, TableDef, - }, - table_operator::TableOperator, -}; +use catalog::table_operator::TableOperator; use ceresdbproto::meta_event::{ meta_event_service_server::MetaEventService, ChangeShardRoleRequest, ChangeShardRoleResponse, CloseShardRequest, CloseShardResponse, CloseTableOnShardRequest, CloseTableOnShardResponse, @@ -21,7 +15,15 @@ use ceresdbproto::meta_event::{ OpenShardResponse, OpenTableOnShardRequest, OpenTableOnShardResponse, SplitShardRequest, SplitShardResponse, }; -use cluster::{shard_set::UpdatedTableInfo, ClusterRef}; +use cluster::{ + shard_operation::{WalCloserAdapter, WalRegionCloserRef}, + shard_operator::{ + CloseContext, CloseTableContext, CreateTableContext, DropTableContext, OpenContext, + OpenTableContext, + }, + shard_set::UpdatedTableInfo, + ClusterRef, +}; use common_types::{schema::SchemaEncoder, table::ShardId}; use common_util::{error::BoxError, runtime::Runtime, time::InstantExt}; use log::{error, info, warn}; @@ -30,24 +32,15 @@ use paste::paste; use proxy::instance::InstanceRef; use query_engine::executor::Executor as QueryExecutor; use snafu::{OptionExt, ResultExt}; -use table_engine::{ - engine::{TableEngineRef, TableState}, - table::TableId, - ANALYTIC_ENGINE_TYPE, -}; +use table_engine::{engine::TableEngineRef, ANALYTIC_ENGINE_TYPE}; use tonic::Response; -use self::shard_operation::WalCloserAdapter; use crate::grpc::{ - meta_event_service::{ - error::{ErrNoCause, ErrWithCause, Result, StatusCode}, - shard_operation::WalRegionCloserRef, - }, + meta_event_service::error::{ErrNoCause, ErrWithCause, Result, StatusCode}, metrics::META_EVENT_GRPC_HANDLER_DURATION_HISTOGRAM_VEC, }; mod error; -mod shard_operation; macro_rules! extract_updated_table_info { ($request: expr) => {{ @@ -284,7 +277,6 @@ async fn do_open_shard(ctx: HandlerContext, shard_info: ShardInfo) -> Result<()> // Try to lock the shard in node level. ctx.acquire_shard_lock(shard_info.id).await?; - // Success to lock the shard in this node, fetch and open shard now. let shard = ctx .cluster .open_shard(&shard_info) @@ -295,44 +287,17 @@ async fn do_open_shard(ctx: HandlerContext, shard_info: ShardInfo) -> Result<()> msg: "fail to open shards in cluster", })?; - // Lock the shard in local, and then recover it. - let _guard = shard.serializing_lock.lock().await; - info!("Shard open sequentially begin, shard_id:{}", shard_info.id); - - let catalog_name = &ctx.default_catalog; - let shard_info = shard.data.shard_info(); - let tables = shard.data.all_tables(); - let table_defs = tables - .into_iter() - .map(|info| TableDef { - catalog_name: catalog_name.clone(), - schema_name: info.schema_name, - id: TableId::from(info.id), - name: info.name, - }) - .collect(); - - let open_shard_request = catalog::schema::OpenShardRequest { - shard_id: shard_info.id, - table_defs, + let open_ctx = OpenContext { + catalog: ctx.default_catalog.clone(), + table_engine: ctx.table_engine.clone(), + table_operator: ctx.table_operator.clone(), engine: ANALYTIC_ENGINE_TYPE.to_string(), }; - let opts = OpenOptions { - table_engine: ctx.table_engine, - }; - - ctx.table_operator - .open_shard(open_shard_request, opts) - .await - .box_err() - .context(ErrWithCause { - code: StatusCode::Internal, - msg: "failed to open shard", - })?; - - info!("Shard open sequentially finish, shard_id:{}", shard_info.id); - Ok(()) + shard.open(open_ctx).await.box_err().context(ErrWithCause { + code: StatusCode::Internal, + msg: "fail to open shard in cluster", + }) } // TODO: maybe we should encapsulate the logic of handling meta event into a @@ -368,50 +333,21 @@ async fn do_close_shard(ctx: &HandlerContext, shard_id: ShardId) -> Result<()> { msg: format!("shard not found when closing shard, shard_id:{shard_id}",), })?; - // Lock for serializing the write operation. - let _guard = shard.serializing_lock.lock().await; - info!("Shard close sequentially begin, shard_id:{shard_id}"); - - shard.data.freeze(); - info!("Shard is frozen before closed, shard_id:{shard_id}"); - - let catalog_name = &ctx.default_catalog.clone(); - let shard_info = shard.data.shard_info(); - let tables = shard.data.all_tables(); - let table_defs = tables - .into_iter() - .map(|info| TableDef { - catalog_name: catalog_name.clone(), - schema_name: info.schema_name, - id: TableId::from(info.id), - name: info.name, - }) - .collect(); - let close_shard_request = catalog::schema::CloseShardRequest { - shard_id: shard_info.id, - table_defs, - engine: ANALYTIC_ENGINE_TYPE.to_string(), - }; - let opts = CloseOptions { + let close_ctx = CloseContext { + catalog: ctx.default_catalog.clone(), table_engine: ctx.table_engine.clone(), + table_operator: ctx.table_operator.clone(), + wal_region_closer: ctx.wal_region_closer.clone(), + engine: ANALYTIC_ENGINE_TYPE.to_string(), }; - ctx.table_operator - .close_shard(close_shard_request, opts) + shard + .close(close_ctx) .await .box_err() .context(ErrWithCause { code: StatusCode::Internal, - msg: "failed to close shard", - })?; - - // Try to close wal region - ctx.wal_region_closer - .close_region(shard_id) - .await - .with_context(|| ErrWithCause { - code: StatusCode::Internal, - msg: format!("fail to close wal region, shard_id:{shard_id}"), + msg: "fail to close shard in cluster", })?; // Remove the shard from the cluster topology after the shard is closed indeed. @@ -457,8 +393,6 @@ async fn handle_create_table_on_shard( request: CreateTableOnShardRequest, ) -> Result<()> { let updated_table_info = extract_updated_table_info!(request); - let shard_info = updated_table_info.shard_info.clone(); - let table_info = updated_table_info.table_info.clone(); let shard = ctx .cluster @@ -471,29 +405,6 @@ async fn handle_create_table_on_shard( ), })?; - // Lock for serializing the write operation. - let _guard = shard.serializing_lock.lock().await; - info!( - "Shard create table sequentially begin, shard_id:{}, table_info:{table_info:?}", - shard_info.id - ); - - // FIXME: maybe should insert table from cluster after having created table. - shard - .data - .try_insert_table(updated_table_info) - .box_err() - .with_context(|| ErrWithCause { - code: StatusCode::Internal, - msg: format!( - "fail to insert table to cluster when creating table on shard, req:{:?}", - request.clone() - ), - })?; - - // Create the table by operator afterwards. - let catalog_name = &ctx.default_catalog; - // Get information for partition table creating. let table_schema = SchemaEncoder::default() .decode(&request.encoded_schema) @@ -506,46 +417,26 @@ async fn handle_create_table_on_shard( ), })?; - let (table_engine, partition_info) = match table_info.partition_info.clone() { - Some(v) => (ctx.partition_table_engine.clone(), Some(v)), - None => (ctx.table_engine.clone(), None), - }; - - // Build create table request and options. - let create_table_request = CreateTableRequest { - catalog_name: catalog_name.clone(), - schema_name: table_info.schema_name.clone(), - table_name: table_info.name.clone(), - table_id: Some(TableId::new(table_info.id)), + let create_table_ctx = CreateTableContext { + catalog: ctx.default_catalog.clone(), + table_engine: ctx.table_engine.clone(), + table_operator: ctx.table_operator.clone(), + partition_table_engine: ctx.partition_table_engine.clone(), + updated_table_info, table_schema, - engine: request.engine, options: request.options, - state: TableState::Stable, - shard_id: shard_info.id, - partition_info, - }; - - let create_opts = CreateOptions { - table_engine, - create_if_not_exists: request.create_if_not_exist, + create_if_not_exist: request.create_if_not_exist, + engine: request.engine, }; - let _ = ctx - .table_operator - .create_table_on_shard(create_table_request.clone(), create_opts) + shard + .create_table(create_table_ctx) .await .box_err() .context(ErrWithCause { code: StatusCode::Internal, - msg: format!("fail to create table with request:{create_table_request:?}"), - })?; - - info!( - "Shard create table sequentially begin, shard_id:{}, table_info:{table_info:?}", - shard_info.id - ); - - Ok(()) + msg: "fail to create table on shard in cluster", + }) } async fn handle_drop_table_on_shard( @@ -553,8 +444,6 @@ async fn handle_drop_table_on_shard( request: DropTableOnShardRequest, ) -> Result<()> { let updated_table_info = extract_updated_table_info!(request); - let shard_info = updated_table_info.shard_info.clone(); - let table_info = updated_table_info.table_info.clone(); let shard = ctx .cluster @@ -567,54 +456,22 @@ async fn handle_drop_table_on_shard( ), })?; - // Lock for serializing the write operation. - let _guard = shard.serializing_lock.lock().await; - info!( - "Shard drop table sequentially begin, shard_id:{}, table_info:{table_info:?}", - shard_info.id - ); - - // FIXME: maybe should insert table from cluster after having dropped table. - shard - .data - .try_remove_table(updated_table_info) - .box_err() - .with_context(|| ErrWithCause { - code: StatusCode::Internal, - msg: format!( - "fail to remove table to cluster when dropping table on shard, req:{:?}", - request.clone() - ), - })?; - - // Drop the table by operator afterwards. - let catalog_name = ctx.default_catalog.clone(); - let drop_table_request = DropTableRequest { - catalog_name, - schema_name: table_info.schema_name.clone(), - table_name: table_info.name.clone(), - // FIXME: the engine type should not use the default one. + let drop_table_ctx = DropTableContext { + catalog: ctx.default_catalog.clone(), + table_engine: ctx.table_engine.clone(), + table_operator: ctx.table_operator.clone(), + updated_table_info, engine: ANALYTIC_ENGINE_TYPE.to_string(), }; - let drop_opts = DropOptions { - table_engine: ctx.table_engine, - }; - ctx.table_operator - .drop_table_on_shard(drop_table_request.clone(), drop_opts) + shard + .drop_table(drop_table_ctx) .await .box_err() - .with_context(|| ErrWithCause { + .context(ErrWithCause { code: StatusCode::Internal, - msg: format!("fail to drop table with request:{drop_table_request:?}"), - })?; - - info!( - "Shard drop table sequentially finish, shard_id:{}, table_info:{table_info:?}", - shard_info.id - ); - - Ok(()) + msg: "fail to drop table on shard in cluster", + }) } async fn handle_open_table_on_shard( @@ -622,8 +479,6 @@ async fn handle_open_table_on_shard( request: OpenTableOnShardRequest, ) -> Result<()> { let updated_table_info = extract_updated_table_info!(request); - let shard_info = updated_table_info.shard_info.clone(); - let table_info = updated_table_info.table_info.clone(); let shard = ctx .cluster @@ -633,55 +488,22 @@ async fn handle_open_table_on_shard( msg: format!("shard not found when opening table on shard, request:{request:?}",), })?; - // Lock for serializing the write operation. - let _guard = shard.serializing_lock.lock().await; - info!( - "Shard open table sequentially begin, shard_id:{}, table_info:{table_info:?}", - shard_info.id - ); - - // FIXME: maybe should insert table from cluster after having opened table. - shard - .data - .try_insert_table(updated_table_info) - .box_err() - .with_context(|| ErrWithCause { - code: StatusCode::Internal, - msg: format!( - "fail to insert table to cluster when opening table on shard, request:{request:?}", - ), - })?; - - // Open the table by operator afterwards. - let catalog_name = ctx.default_catalog.clone(); - let open_table_request = OpenTableRequest { - catalog_name, - schema_name: table_info.schema_name.clone(), - table_name: table_info.name.clone(), - // FIXME: the engine type should not use the default one. + let open_table_ctx = OpenTableContext { + catalog: ctx.default_catalog.clone(), + table_engine: ctx.table_engine.clone(), + table_operator: ctx.table_operator.clone(), + updated_table_info, engine: ANALYTIC_ENGINE_TYPE.to_string(), - shard_id: shard_info.id, - table_id: TableId::new(table_info.id), - }; - let open_opts = OpenOptions { - table_engine: ctx.table_engine, }; - ctx.table_operator - .open_table_on_shard(open_table_request.clone(), open_opts) + shard + .open_table(open_table_ctx) .await .box_err() - .with_context(|| ErrWithCause { + .context(ErrWithCause { code: StatusCode::Internal, - msg: format!("fail to open table with request:{open_table_request:?}"), - })?; - - info!( - "Shard open table sequentially finish, shard_id:{}, table_info:{table_info:?}", - shard_info.id - ); - - Ok(()) + msg: "fail to open table on shard in cluster", + }) } async fn handle_close_table_on_shard( @@ -690,7 +512,6 @@ async fn handle_close_table_on_shard( ) -> Result<()> { let updated_table_info = extract_updated_table_info!(request); let shard_id = updated_table_info.shard_info.id; - let table_info = updated_table_info.table_info.clone(); let shard = ctx .cluster @@ -703,49 +524,22 @@ async fn handle_close_table_on_shard( ), })?; - // Lock for serializing the write operation. - let _guard = shard.serializing_lock.lock().await; - info!("Shard close table sequentially begin, shard_id:{shard_id}, table_info:{table_info:?}"); - - // FIXME: maybe should remove table from cluster after having closed table. - shard - .data - .try_remove_table(updated_table_info) - .box_err() - .with_context(|| ErrWithCause { - code: StatusCode::Internal, - msg: format!( - "fail to remove table from cluster when closing table on shard, req:{:?}", - request.clone() - ), - })?; - - // Close the table by catalog manager afterwards. - let catalog_name = &ctx.default_catalog; - let close_table_request = CloseTableRequest { - catalog_name: catalog_name.clone(), - schema_name: table_info.schema_name.clone(), - table_name: table_info.name.clone(), - table_id: TableId::new(table_info.id), - // FIXME: the engine type should not use the default one. + let close_table_ctx = CloseTableContext { + catalog: ctx.default_catalog.clone(), + table_engine: ctx.table_engine.clone(), + table_operator: ctx.table_operator.clone(), + updated_table_info, engine: ANALYTIC_ENGINE_TYPE.to_string(), }; - let close_opts = CloseOptions { - table_engine: ctx.table_engine, - }; - ctx.table_operator - .close_table_on_shard(close_table_request.clone(), close_opts) + shard + .close_table(close_table_ctx) .await .box_err() - .with_context(|| ErrWithCause { + .context(ErrWithCause { code: StatusCode::Internal, - msg: format!("fail to close table with request:{close_table_request:?}"), - })?; - - info!("Shard close table sequentially finish, shard_id:{shard_id}, table_info:{table_info:?}"); - - Ok(()) + msg: "fail to close table on shard in cluster", + }) } #[async_trait] From 6c9be938091e6a9608dde1037b54ddb3cd0444eb Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 10 Jul 2023 20:35:55 +0800 Subject: [PATCH 12/15] rename `get_shard` to `shard` in `Cluster`. --- cluster/src/cluster_impl.rs | 6 ++-- cluster/src/lib.rs | 2 +- router/src/cluster_based.rs | 2 +- server/src/grpc/meta_event_service/mod.rs | 34 ++++++++++------------- 4 files changed, 19 insertions(+), 25 deletions(-) diff --git a/cluster/src/cluster_impl.rs b/cluster/src/cluster_impl.rs index ce90b3b16c..549ef330bc 100644 --- a/cluster/src/cluster_impl.rs +++ b/cluster/src/cluster_impl.rs @@ -287,7 +287,7 @@ impl Inner { Ok(shard) } - fn get_shard(&self, shard_id: ShardId) -> Option { + fn shard(&self, shard_id: ShardId) -> Option { self.shard_set.get(shard_id) } @@ -337,8 +337,8 @@ impl Cluster for ClusterImpl { self.inner.open_shard(shard_info).await } - fn get_shard(&self, shard_id: ShardId) -> Option { - self.inner.get_shard(shard_id) + fn shard(&self, shard_id: ShardId) -> Option { + self.inner.shard(shard_id) } async fn close_shard(&self, shard_id: ShardId) -> Result { diff --git a/cluster/src/lib.rs b/cluster/src/lib.rs index cf3dfd1dea..ed6fc34ee9 100644 --- a/cluster/src/lib.rs +++ b/cluster/src/lib.rs @@ -163,7 +163,7 @@ pub trait Cluster { /// /// If target shard has opened in cluster, return it. Otherwise, return /// None. - fn get_shard(&self, shard_id: ShardId) -> Option; + fn shard(&self, shard_id: ShardId) -> Option; /// Close shard. /// diff --git a/router/src/cluster_based.rs b/router/src/cluster_based.rs index 3602c16622..78a6ab185b 100644 --- a/router/src/cluster_based.rs +++ b/router/src/cluster_based.rs @@ -190,7 +190,7 @@ mod tests { unimplemented!(); } - fn get_shard(&self, _: ShardId) -> Option { + fn shard(&self, _: ShardId) -> Option { unimplemented!(); } diff --git a/server/src/grpc/meta_event_service/mod.rs b/server/src/grpc/meta_event_service/mod.rs index 42d575c591..312dcfb0ae 100644 --- a/server/src/grpc/meta_event_service/mod.rs +++ b/server/src/grpc/meta_event_service/mod.rs @@ -325,13 +325,10 @@ async fn handle_open_shard(ctx: HandlerContext, request: OpenShardRequest) -> Re } async fn do_close_shard(ctx: &HandlerContext, shard_id: ShardId) -> Result<()> { - let shard = ctx - .cluster - .get_shard(shard_id) - .with_context(|| ErrNoCause { - code: StatusCode::Internal, - msg: format!("shard not found when closing shard, shard_id:{shard_id}",), - })?; + let shard = ctx.cluster.shard(shard_id).with_context(|| ErrNoCause { + code: StatusCode::Internal, + msg: format!("shard not found when closing shard, shard_id:{shard_id}",), + })?; let close_ctx = CloseContext { catalog: ctx.default_catalog.clone(), @@ -396,7 +393,7 @@ async fn handle_create_table_on_shard( let shard = ctx .cluster - .get_shard(updated_table_info.shard_info.id) + .shard(updated_table_info.shard_info.id) .with_context(|| ErrNoCause { code: StatusCode::Internal, msg: format!( @@ -447,7 +444,7 @@ async fn handle_drop_table_on_shard( let shard = ctx .cluster - .get_shard(updated_table_info.shard_info.id) + .shard(updated_table_info.shard_info.id) .with_context(|| ErrNoCause { code: StatusCode::Internal, msg: format!( @@ -482,7 +479,7 @@ async fn handle_open_table_on_shard( let shard = ctx .cluster - .get_shard(updated_table_info.shard_info.id) + .shard(updated_table_info.shard_info.id) .with_context(|| ErrNoCause { code: StatusCode::Internal, msg: format!("shard not found when opening table on shard, request:{request:?}",), @@ -513,16 +510,13 @@ async fn handle_close_table_on_shard( let updated_table_info = extract_updated_table_info!(request); let shard_id = updated_table_info.shard_info.id; - let shard = ctx - .cluster - .get_shard(shard_id) - .with_context(|| ErrNoCause { - code: StatusCode::Internal, - msg: format!( - "shard not found when closing table on shard, req:{:?}", - request.clone() - ), - })?; + let shard = ctx.cluster.shard(shard_id).with_context(|| ErrNoCause { + code: StatusCode::Internal, + msg: format!( + "shard not found when closing table on shard, req:{:?}", + request.clone() + ), + })?; let close_table_ctx = CloseTableContext { catalog: ctx.default_catalog.clone(), From a5db61927b00ad2dcee210fcbbade6daa7123aec Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 10 Jul 2023 20:41:13 +0800 Subject: [PATCH 13/15] fix DropTableRequest. --- cluster/src/shard_operator.rs | 3 +-- server/src/grpc/meta_event_service/mod.rs | 5 +++++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/cluster/src/shard_operator.rs b/cluster/src/shard_operator.rs index 4d039d39f2..129715b702 100644 --- a/cluster/src/shard_operator.rs +++ b/cluster/src/shard_operator.rs @@ -293,8 +293,7 @@ impl ShardOperator { catalog_name: ctx.catalog, schema_name: table_info.schema_name.clone(), table_name: table_info.name.clone(), - // FIXME: the engine type should not use the default one. - engine: ANALYTIC_ENGINE_TYPE.to_string(), + engine: ctx.engine, }; let drop_opts = DropOptions { table_engine: ctx.table_engine, diff --git a/server/src/grpc/meta_event_service/mod.rs b/server/src/grpc/meta_event_service/mod.rs index 312dcfb0ae..f800d3f3d9 100644 --- a/server/src/grpc/meta_event_service/mod.rs +++ b/server/src/grpc/meta_event_service/mod.rs @@ -291,6 +291,7 @@ async fn do_open_shard(ctx: HandlerContext, shard_info: ShardInfo) -> Result<()> catalog: ctx.default_catalog.clone(), table_engine: ctx.table_engine.clone(), table_operator: ctx.table_operator.clone(), + // FIXME: the engine type should not use the default one. engine: ANALYTIC_ENGINE_TYPE.to_string(), }; @@ -335,6 +336,7 @@ async fn do_close_shard(ctx: &HandlerContext, shard_id: ShardId) -> Result<()> { table_engine: ctx.table_engine.clone(), table_operator: ctx.table_operator.clone(), wal_region_closer: ctx.wal_region_closer.clone(), + // FIXME: the engine type should not use the default one. engine: ANALYTIC_ENGINE_TYPE.to_string(), }; @@ -458,6 +460,7 @@ async fn handle_drop_table_on_shard( table_engine: ctx.table_engine.clone(), table_operator: ctx.table_operator.clone(), updated_table_info, + // FIXME: the engine type should not use the default one. engine: ANALYTIC_ENGINE_TYPE.to_string(), }; @@ -490,6 +493,7 @@ async fn handle_open_table_on_shard( table_engine: ctx.table_engine.clone(), table_operator: ctx.table_operator.clone(), updated_table_info, + // FIXME: the engine type should not use the default one. engine: ANALYTIC_ENGINE_TYPE.to_string(), }; @@ -523,6 +527,7 @@ async fn handle_close_table_on_shard( table_engine: ctx.table_engine.clone(), table_operator: ctx.table_operator.clone(), updated_table_info, + // FIXME: the engine type should not use the default one. engine: ANALYTIC_ENGINE_TYPE.to_string(), }; From 90a98861344e69f85100aff80660f83f3e5b1110 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 10 Jul 2023 20:47:29 +0800 Subject: [PATCH 14/15] add comments. --- cluster/src/shard_operator.rs | 1 - cluster/src/shard_set.rs | 15 +++++++-------- server/src/grpc/meta_event_service/mod.rs | 12 ++++++------ 3 files changed, 13 insertions(+), 15 deletions(-) diff --git a/cluster/src/shard_operator.rs b/cluster/src/shard_operator.rs index 129715b702..fab9a1add7 100644 --- a/cluster/src/shard_operator.rs +++ b/cluster/src/shard_operator.rs @@ -15,7 +15,6 @@ use snafu::ResultExt; use table_engine::{ engine::{TableEngineRef, TableState}, table::TableId, - ANALYTIC_ENGINE_TYPE, }; use crate::{ diff --git a/cluster/src/shard_set.rs b/cluster/src/shard_set.rs index a537964ffe..f1921900fb 100644 --- a/cluster/src/shard_set.rs +++ b/cluster/src/shard_set.rs @@ -13,8 +13,9 @@ use crate::{ Result, ShardVersionMismatch, TableAlreadyExists, TableNotFound, UpdateFrozenShard, }; -/// [ShardTablesCache] caches the information about tables and shards, and the -/// relationship between them is: one shard -> multiple tables. +/// Shard set +/// +/// Manage all shards opened on current node #[derive(Debug, Default, Clone)] pub struct ShardSet { inner: Arc>>, @@ -46,6 +47,9 @@ impl ShardSet { } } +/// Shard +/// +/// NOTICE: all write operations on a shard will be performed sequentially. pub struct Shard { pub data: ShardDataRef, pub operator: tokio::sync::Mutex, @@ -113,12 +117,6 @@ impl Shard { pub type ShardRef = Arc; -#[derive(Debug, Clone)] -pub struct TableWithShards { - pub table_info: TableInfo, - pub shard_infos: Vec, -} - #[derive(Debug, Clone)] pub struct UpdatedTableInfo { pub prev_version: u64, @@ -126,6 +124,7 @@ pub struct UpdatedTableInfo { pub table_info: TableInfo, } +/// Shard data #[derive(Debug)] pub struct ShardData { /// Shard info diff --git a/server/src/grpc/meta_event_service/mod.rs b/server/src/grpc/meta_event_service/mod.rs index f800d3f3d9..165f15b4c6 100644 --- a/server/src/grpc/meta_event_service/mod.rs +++ b/server/src/grpc/meta_event_service/mod.rs @@ -297,7 +297,7 @@ async fn do_open_shard(ctx: HandlerContext, shard_info: ShardInfo) -> Result<()> shard.open(open_ctx).await.box_err().context(ErrWithCause { code: StatusCode::Internal, - msg: "fail to open shard in cluster", + msg: "fail to open shard", }) } @@ -346,7 +346,7 @@ async fn do_close_shard(ctx: &HandlerContext, shard_id: ShardId) -> Result<()> { .box_err() .context(ErrWithCause { code: StatusCode::Internal, - msg: "fail to close shard in cluster", + msg: "fail to close shard", })?; // Remove the shard from the cluster topology after the shard is closed indeed. @@ -434,7 +434,7 @@ async fn handle_create_table_on_shard( .box_err() .context(ErrWithCause { code: StatusCode::Internal, - msg: "fail to create table on shard in cluster", + msg: "fail to create table on shard", }) } @@ -470,7 +470,7 @@ async fn handle_drop_table_on_shard( .box_err() .context(ErrWithCause { code: StatusCode::Internal, - msg: "fail to drop table on shard in cluster", + msg: "fail to drop table on shard", }) } @@ -503,7 +503,7 @@ async fn handle_open_table_on_shard( .box_err() .context(ErrWithCause { code: StatusCode::Internal, - msg: "fail to open table on shard in cluster", + msg: "fail to open table on shard", }) } @@ -537,7 +537,7 @@ async fn handle_close_table_on_shard( .box_err() .context(ErrWithCause { code: StatusCode::Internal, - msg: "fail to close table on shard in cluster", + msg: "fail to close table on shard", }) } From f8c38edc8f1f1cd2f0b1dbc25b085b656c9fcee5 Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 11 Jul 2023 10:22:59 +0800 Subject: [PATCH 15/15] remove unnecessary clone, make fields in `Shard` private. --- cluster/src/shard_operator.rs | 5 +---- cluster/src/shard_set.rs | 4 ++-- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/cluster/src/shard_operator.rs b/cluster/src/shard_operator.rs index fab9a1add7..76e3189bbc 100644 --- a/cluster/src/shard_operator.rs +++ b/cluster/src/shard_operator.rs @@ -118,7 +118,6 @@ impl ShardOperator { (shard_info, tables) }; - // Lock the shard in local, and then recover it. info!("ShardOperator open sequentially begin, shard_id:{shard_info:?}"); let table_defs = tables @@ -227,8 +226,6 @@ impl ShardOperator { } // Create the table by operator afterwards. - let catalog_name = ctx.catalog; - let (table_engine, partition_info) = match table_info.partition_info.clone() { Some(v) => (ctx.partition_table_engine.clone(), Some(v)), None => (ctx.table_engine.clone(), None), @@ -236,7 +233,7 @@ impl ShardOperator { // Build create table request and options. let create_table_request = CreateTableRequest { - catalog_name: catalog_name.clone(), + catalog_name: ctx.catalog, schema_name: table_info.schema_name.clone(), table_name: table_info.name.clone(), table_id: Some(TableId::new(table_info.id)), diff --git a/cluster/src/shard_set.rs b/cluster/src/shard_set.rs index f1921900fb..28b57566ea 100644 --- a/cluster/src/shard_set.rs +++ b/cluster/src/shard_set.rs @@ -51,8 +51,8 @@ impl ShardSet { /// /// NOTICE: all write operations on a shard will be performed sequentially. pub struct Shard { - pub data: ShardDataRef, - pub operator: tokio::sync::Mutex, + data: ShardDataRef, + operator: tokio::sync::Mutex, } impl std::fmt::Debug for Shard {