From 71524074285f49f5173d30eea341779764b1480a Mon Sep 17 00:00:00 2001 From: ZonaHe Date: Thu, 28 Dec 2023 19:51:43 +0800 Subject: [PATCH 01/21] feat: update dashboard to v0.4.5 (#3033) Co-authored-by: ZonaHex --- src/servers/dashboard/VERSION | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/servers/dashboard/VERSION b/src/servers/dashboard/VERSION index 79b0815e6a8f..a423f7f06884 100644 --- a/src/servers/dashboard/VERSION +++ b/src/servers/dashboard/VERSION @@ -1 +1 @@ -v0.4.4 +v0.4.5 From b526d159c3dd3bb9309720eadc96464ddb2e9b56 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Fri, 29 Dec 2023 01:12:07 +0900 Subject: [PATCH 02/21] fix: replay memtable should from `flushed_entry_id + 1` (#3038) * fix: replay memtable should from flushed_entry_id + 1 * chore: apply suggestions from CR --- src/mito2/src/region/opener.rs | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/mito2/src/region/opener.rs b/src/mito2/src/region/opener.rs index 7b969d578d00..9fd6e36dc898 100644 --- a/src/mito2/src/region/opener.rs +++ b/src/mito2/src/region/opener.rs @@ -257,8 +257,9 @@ impl RegionOpener { let version_control = Arc::new(VersionControl::new(version)); if !self.skip_wal_replay { info!( - "Start replaying memtable at flushed_entry_id {} for region {}", - flushed_entry_id, region_id + "Start replaying memtable at flushed_entry_id + 1 {} for region {}", + flushed_entry_id + 1, + region_id ); replay_memtable( wal, @@ -380,9 +381,12 @@ pub(crate) async fn replay_memtable( // data in the WAL. let mut last_entry_id = flushed_entry_id; let mut region_write_ctx = RegionWriteCtx::new(region_id, version_control, wal_options.clone()); - let mut wal_stream = wal.scan(region_id, flushed_entry_id, wal_options)?; + + let replay_from_entry_id = flushed_entry_id + 1; + let mut wal_stream = wal.scan(region_id, replay_from_entry_id, wal_options)?; while let Some(res) = wal_stream.next().await { let (entry_id, entry) = res?; + debug_assert!(entry_id > flushed_entry_id); last_entry_id = last_entry_id.max(entry_id); for mutation in entry.mutations { rows_replayed += mutation From d22072f68be418e6d3be62166caee2c152f86e8a Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Fri, 29 Dec 2023 15:57:00 +0900 Subject: [PATCH 03/21] feat: expose region migration http endpoint (#3032) * feat: add region migration endpoint * feat: implement naive peer registry * chore: apply suggestions from CR * chore: rename `ContextFactoryImpl` to `DefaultContextFactory` * chore: rename unregister to deregister * refactor: use lease-based alive datanode checking --- src/meta-srv/src/cluster.rs | 13 +++- src/meta-srv/src/error.rs | 6 +- src/meta-srv/src/handler.rs | 2 +- src/meta-srv/src/lease.rs | 42 +++++++++-- src/meta-srv/src/metasrv.rs | 6 ++ src/meta-srv/src/metasrv/builder.rs | 14 ++++ .../src/procedure/region_migration.rs | 22 +++++- .../src/procedure/region_migration/manager.rs | 29 ++++--- .../procedure/region_migration/test_util.rs | 6 +- src/meta-srv/src/service/admin.rs | 6 ++ .../src/service/admin/region_migration.rs | 75 ++++++++++++++++--- src/meta-srv/src/service/heartbeat.rs | 2 +- 12 files changed, 185 insertions(+), 38 deletions(-) diff --git a/src/meta-srv/src/cluster.rs b/src/meta-srv/src/cluster.rs index 83a5dd984cd0..7e460664c181 100644 --- a/src/meta-srv/src/cluster.rs +++ b/src/meta-srv/src/cluster.rs @@ -79,6 +79,17 @@ impl MetaPeerClient { to_stat_kv_map(kvs) } + // Get kv information from the leader's in_mem kv store. + pub async fn get(&self, key: Vec) -> Result> { + let mut kvs = self.range(key, vec![], false).await?; + Ok(if kvs.is_empty() { + None + } else { + debug_assert_eq!(kvs.len(), 1); + Some(kvs.remove(0)) + }) + } + // Range kv information from the leader's in_mem kv store pub async fn range( &self, @@ -228,7 +239,7 @@ impl MetaPeerClient { // Check if the meta node is a leader node. // Note: when self.election is None, we also consider the meta node is leader - fn is_leader(&self) -> bool { + pub(crate) fn is_leader(&self) -> bool { self.election .as_ref() .map(|election| election.is_leader()) diff --git a/src/meta-srv/src/error.rs b/src/meta-srv/src/error.rs index 92d7249e33ca..530fba83aa2e 100644 --- a/src/meta-srv/src/error.rs +++ b/src/meta-srv/src/error.rs @@ -32,6 +32,9 @@ use crate::pubsub::Message; #[snafu(visibility(pub))] #[stack_trace_debug] pub enum Error { + #[snafu(display("The target peer is unavailable temporally: {}", peer_id))] + PeerUnavailable { location: Location, peer_id: u64 }, + #[snafu(display("Another migration procedure is running for region: {}", region_id))] MigrationRunning { location: Location, @@ -650,7 +653,8 @@ impl ErrorExt for Error { | Error::Join { .. } | Error::WeightArray { .. } | Error::NotSetWeightArray { .. } - | Error::Unsupported { .. } => StatusCode::Internal, + | Error::Unsupported { .. } + | Error::PeerUnavailable { .. } => StatusCode::Internal, Error::TableAlreadyExists { .. } => StatusCode::TableAlreadyExists, Error::EmptyKey { .. } | Error::MissingRequiredParameter { .. } diff --git a/src/meta-srv/src/handler.rs b/src/meta-srv/src/handler.rs index 8be541ee68bc..4de3d5530dd8 100644 --- a/src/meta-srv/src/handler.rs +++ b/src/meta-srv/src/handler.rs @@ -229,7 +229,7 @@ impl HeartbeatHandlerGroup { let _ = self.pushers.insert(key.to_string(), pusher).await; } - pub async fn unregister(&self, key: impl AsRef) -> Option { + pub async fn deregister(&self, key: impl AsRef) -> Option { let key = key.as_ref(); METRIC_META_HEARTBEAT_CONNECTION_NUM.dec(); info!("Pusher unregister: {}", key); diff --git a/src/meta-srv/src/lease.rs b/src/meta-srv/src/lease.rs index 0f8c409406c9..2fa3224e35b4 100644 --- a/src/meta-srv/src/lease.rs +++ b/src/meta-srv/src/lease.rs @@ -14,27 +14,57 @@ use std::collections::HashMap; -use common_meta::util; +use common_meta::peer::Peer; +use common_meta::{util, ClusterId}; use common_time::util as time_util; use crate::cluster::MetaPeerClientRef; use crate::error::Result; use crate::keys::{LeaseKey, LeaseValue, DN_LEASE_PREFIX}; +fn build_lease_filter(lease_secs: u64) -> impl Fn(&LeaseKey, &LeaseValue) -> bool { + move |_: &LeaseKey, v: &LeaseValue| { + ((time_util::current_time_millis() - v.timestamp_millis) as u64) < lease_secs * 1000 + } +} + +pub async fn lookup_alive_datanode_peer( + cluster_id: ClusterId, + datanode_id: u64, + meta_peer_client: &MetaPeerClientRef, + lease_secs: u64, +) -> Result> { + let lease_filter = build_lease_filter(lease_secs); + let lease_key = LeaseKey { + cluster_id, + node_id: datanode_id, + }; + let Some(kv) = meta_peer_client.get(lease_key.clone().try_into()?).await? else { + return Ok(None); + }; + let lease_value: LeaseValue = kv.value.try_into()?; + if lease_filter(&lease_key, &lease_value) { + Ok(Some(Peer { + id: lease_key.node_id, + addr: lease_value.node_addr, + })) + } else { + Ok(None) + } +} + pub async fn alive_datanodes( - cluster_id: u64, + cluster_id: ClusterId, meta_peer_client: &MetaPeerClientRef, lease_secs: u64, ) -> Result> { - let lease_filter = |_: &LeaseKey, v: &LeaseValue| { - ((time_util::current_time_millis() - v.timestamp_millis) as u64) < lease_secs * 1000 - }; + let lease_filter = build_lease_filter(lease_secs); filter_datanodes(cluster_id, meta_peer_client, lease_filter).await } pub async fn filter_datanodes

( - cluster_id: u64, + cluster_id: ClusterId, meta_peer_client: &MetaPeerClientRef, predicate: P, ) -> Result> diff --git a/src/meta-srv/src/metasrv.rs b/src/meta-srv/src/metasrv.rs index 4d6782a2b630..dba3c4485002 100644 --- a/src/meta-srv/src/metasrv.rs +++ b/src/meta-srv/src/metasrv.rs @@ -48,6 +48,7 @@ use crate::error::{ use crate::failure_detector::PhiAccrualFailureDetectorOptions; use crate::handler::HeartbeatHandlerGroup; use crate::lock::DistLockRef; +use crate::procedure::region_migration::manager::RegionMigrationManagerRef; use crate::pubsub::{PublishRef, SubscribeManagerRef}; use crate::selector::{Selector, SelectorType}; use crate::service::mailbox::MailboxRef; @@ -249,6 +250,7 @@ pub struct MetaSrv { table_metadata_manager: TableMetadataManagerRef, memory_region_keeper: MemoryRegionKeeperRef, greptimedb_telemetry_task: Arc, + region_migration_manager: RegionMigrationManagerRef, plugins: Plugins, } @@ -411,6 +413,10 @@ impl MetaSrv { &self.memory_region_keeper } + pub fn region_migration_manager(&self) -> &RegionMigrationManagerRef { + &self.region_migration_manager + } + pub fn publish(&self) -> Option { self.plugins.get::() } diff --git a/src/meta-srv/src/metasrv/builder.rs b/src/meta-srv/src/metasrv/builder.rs index 105e9dab0017..28a34714d665 100644 --- a/src/meta-srv/src/metasrv/builder.rs +++ b/src/meta-srv/src/metasrv/builder.rs @@ -57,6 +57,8 @@ use crate::metasrv::{ ElectionRef, MetaSrv, MetaSrvOptions, MetasrvInfo, SelectorContext, SelectorRef, TABLE_ID_SEQ, }; use crate::procedure::region_failover::RegionFailoverManager; +use crate::procedure::region_migration::manager::RegionMigrationManager; +use crate::procedure::region_migration::DefaultContextFactory; use crate::pubsub::PublishRef; use crate::selector::lease_based::LeaseBasedSelector; use crate::service::mailbox::MailboxRef; @@ -236,6 +238,17 @@ impl MetaSrvBuilder { &opening_region_keeper, )?; + let region_migration_manager = Arc::new(RegionMigrationManager::new( + procedure_manager.clone(), + DefaultContextFactory::new( + table_metadata_manager.clone(), + opening_region_keeper.clone(), + mailbox.clone(), + options.server_addr.clone(), + ), + )); + region_migration_manager.try_start()?; + let handler_group = match handler_group { Some(handler_group) => handler_group, None => { @@ -323,6 +336,7 @@ impl MetaSrvBuilder { .await, plugins: plugins.unwrap_or_else(Plugins::default), memory_region_keeper: opening_region_keeper, + region_migration_manager, }) } } diff --git a/src/meta-srv/src/procedure/region_migration.rs b/src/meta-srv/src/procedure/region_migration.rs index 5299972236e0..a1e92277d60b 100644 --- a/src/meta-srv/src/procedure/region_migration.rs +++ b/src/meta-srv/src/procedure/region_migration.rs @@ -127,7 +127,7 @@ pub trait ContextFactory { /// Default implementation. #[derive(Clone)] -pub struct ContextFactoryImpl { +pub struct DefaultContextFactory { volatile_ctx: VolatileContext, table_metadata_manager: TableMetadataManagerRef, opening_region_keeper: MemoryRegionKeeperRef, @@ -135,7 +135,25 @@ pub struct ContextFactoryImpl { server_addr: String, } -impl ContextFactory for ContextFactoryImpl { +impl DefaultContextFactory { + /// Returns an [ContextFactoryImpl]. + pub fn new( + table_metadata_manager: TableMetadataManagerRef, + opening_region_keeper: MemoryRegionKeeperRef, + mailbox: MailboxRef, + server_addr: String, + ) -> Self { + Self { + volatile_ctx: VolatileContext::default(), + table_metadata_manager, + opening_region_keeper, + mailbox, + server_addr, + } + } +} + +impl ContextFactory for DefaultContextFactory { fn new_context(self, persistent_ctx: PersistentContext) -> Context { Context { persistent_ctx, diff --git a/src/meta-srv/src/procedure/region_migration/manager.rs b/src/meta-srv/src/procedure/region_migration/manager.rs index cfb125c49ba2..03794ed85d11 100644 --- a/src/meta-srv/src/procedure/region_migration/manager.rs +++ b/src/meta-srv/src/procedure/region_migration/manager.rs @@ -21,21 +21,23 @@ use common_meta::key::table_route::TableRouteValue; use common_meta::peer::Peer; use common_meta::rpc::router::RegionRoute; use common_meta::ClusterId; -use common_procedure::{watcher, ProcedureManagerRef, ProcedureWithId}; +use common_procedure::{watcher, ProcedureId, ProcedureManagerRef, ProcedureWithId}; use common_telemetry::{error, info}; use snafu::{ensure, OptionExt, ResultExt}; use store_api::storage::RegionId; use crate::error::{self, Result}; use crate::procedure::region_migration::{ - ContextFactoryImpl, PersistentContext, RegionMigrationProcedure, + DefaultContextFactory, PersistentContext, RegionMigrationProcedure, }; +pub type RegionMigrationManagerRef = Arc; + /// Manager of region migration procedure. -pub(crate) struct RegionMigrationManager { +pub struct RegionMigrationManager { procedure_manager: ProcedureManagerRef, running_procedures: Arc>>, - context_factory: ContextFactoryImpl, + context_factory: DefaultContextFactory, } /// The guard of running [RegionMigrationProcedureTask]. @@ -55,10 +57,10 @@ impl Drop for RegionMigrationProcedureGuard { #[derive(Debug, Clone)] pub(crate) struct RegionMigrationProcedureTask { - cluster_id: ClusterId, - region_id: RegionId, - from_peer: Peer, - to_peer: Peer, + pub(crate) cluster_id: ClusterId, + pub(crate) region_id: RegionId, + pub(crate) from_peer: Peer, + pub(crate) to_peer: Peer, } impl Display for RegionMigrationProcedureTask { @@ -93,7 +95,7 @@ impl RegionMigrationManager { /// Returns new [RegionMigrationManager] pub(crate) fn new( procedure_manager: ProcedureManagerRef, - context_factory: ContextFactoryImpl, + context_factory: DefaultContextFactory, ) -> Self { Self { procedure_manager, @@ -221,7 +223,10 @@ impl RegionMigrationManager { } /// Submits a new region migration procedure. - pub(crate) async fn submit_procedure(&self, task: RegionMigrationProcedureTask) -> Result<()> { + pub(crate) async fn submit_procedure( + &self, + task: RegionMigrationProcedureTask, + ) -> Result> { let Some(guard) = self.insert_running_procedure(&task) else { return error::MigrationRunningSnafu { region_id: task.region_id, @@ -243,7 +248,7 @@ impl RegionMigrationManager { if self.has_migrated(®ion_route, &task)? { info!("Skipping region migration task: {task}"); - return Ok(()); + return Ok(None); } self.verify_region_leader_peer(®ion_route, &task)?; @@ -274,7 +279,7 @@ impl RegionMigrationManager { info!("Region migration procedure {procedure_id} for {task} is finished successfully!"); }); - Ok(()) + Ok(Some(procedure_id)) } } diff --git a/src/meta-srv/src/procedure/region_migration/test_util.rs b/src/meta-srv/src/procedure/region_migration/test_util.rs index f3a0183f233f..4431791ff70f 100644 --- a/src/meta-srv/src/procedure/region_migration/test_util.rs +++ b/src/meta-srv/src/procedure/region_migration/test_util.rs @@ -43,7 +43,7 @@ use tokio::sync::mpsc::{Receiver, Sender}; use super::migration_abort::RegionMigrationAbort; use super::upgrade_candidate_region::UpgradeCandidateRegion; -use super::{Context, ContextFactory, ContextFactoryImpl, State, VolatileContext}; +use super::{Context, ContextFactory, DefaultContextFactory, State, VolatileContext}; use crate::error::{self, Error, Result}; use crate::handler::{HeartbeatMailbox, Pusher, Pushers}; use crate::procedure::region_migration::downgrade_leader_region::DowngradeLeaderRegion; @@ -120,8 +120,8 @@ impl TestingEnv { } /// Returns a context of region migration procedure. - pub fn context_factory(&self) -> ContextFactoryImpl { - ContextFactoryImpl { + pub fn context_factory(&self) -> DefaultContextFactory { + DefaultContextFactory { table_metadata_manager: self.table_metadata_manager.clone(), opening_region_keeper: self.opening_region_keeper.clone(), volatile_ctx: Default::default(), diff --git a/src/meta-srv/src/service/admin.rs b/src/meta-srv/src/service/admin.rs index 64965571b270..fa1443de32d9 100644 --- a/src/meta-srv/src/service/admin.rs +++ b/src/meta-srv/src/service/admin.rs @@ -93,6 +93,12 @@ pub fn make_admin_service(meta_srv: MetaSrv) -> Admin { .route("/route", handler.clone()) .route("/route/help", handler); + let handler = region_migration::SubmitRegionMigrationTaskHandler { + region_migration_manager: meta_srv.region_migration_manager().clone(), + meta_peer_client: meta_srv.meta_peer_client().clone(), + }; + let router = router.route("/region-migration", handler); + let router = Router::nest("/admin", router); Admin::new(router) diff --git a/src/meta-srv/src/service/admin/region_migration.rs b/src/meta-srv/src/service/admin/region_migration.rs index 0cc1457f5019..544a6d40859a 100644 --- a/src/meta-srv/src/service/admin/region_migration.rs +++ b/src/meta-srv/src/service/admin/region_migration.rs @@ -17,22 +17,24 @@ use std::num::ParseIntError; use std::str::FromStr; use common_meta::peer::Peer; -use common_meta::ClusterId; +use common_meta::{distributed_time_constants, ClusterId}; use serde::Serialize; -use snafu::ResultExt; +use snafu::{ensure, OptionExt, ResultExt}; use store_api::storage::RegionId; use tonic::codegen::http; use super::HttpHandler; +use crate::cluster::MetaPeerClientRef; use crate::error::{self, Error, Result}; - -pub trait PeerLookup: Send + Sync { - fn peer(&self, peer_id: u64) -> Option; -} +use crate::lease::lookup_alive_datanode_peer; +use crate::procedure::region_migration::manager::{ + RegionMigrationManagerRef, RegionMigrationProcedureTask, +}; /// The handler of submitting migration task. pub struct SubmitRegionMigrationTaskHandler { - // TODO(weny): waits for https://github.com/GreptimeTeam/greptimedb/pull/3014 + pub region_migration_manager: RegionMigrationManagerRef, + pub meta_peer_client: MetaPeerClientRef, } #[derive(Debug, Clone, PartialEq, Eq)] @@ -45,7 +47,8 @@ struct SubmitRegionMigrationTaskRequest { #[derive(Debug, Serialize)] struct SubmitRegionMigrationTaskResponse { - procedure_id: String, + /// The `None` stands region has been migrated. + procedure_id: Option, } fn parse_num_parameter_with_default( @@ -96,13 +99,63 @@ impl TryFrom<&HashMap> for SubmitRegionMigrationTaskRequest { } impl SubmitRegionMigrationTaskHandler { + fn is_leader(&self) -> bool { + self.meta_peer_client.is_leader() + } + + /// Checks the peer is available. + async fn lookup_peer(&self, cluster_id: ClusterId, peer_id: u64) -> Result> { + lookup_alive_datanode_peer( + cluster_id, + peer_id, + &self.meta_peer_client, + distributed_time_constants::DATANODE_LEASE_SECS, + ) + .await + } + /// Submits a region migration task, returns the procedure id. async fn handle_submit( &self, - _task: SubmitRegionMigrationTaskRequest, + task: SubmitRegionMigrationTaskRequest, ) -> Result { - // TODO(weny): waits for https://github.com/GreptimeTeam/greptimedb/pull/3014 - todo!() + ensure!( + self.is_leader(), + error::UnexpectedSnafu { + violated: "Trying to submit a region migration procedure to non-leader meta server" + } + ); + + let SubmitRegionMigrationTaskRequest { + cluster_id, + region_id, + from_peer_id, + to_peer_id, + } = task; + + let from_peer = self.lookup_peer(cluster_id, from_peer_id).await?.context( + error::PeerUnavailableSnafu { + peer_id: from_peer_id, + }, + )?; + let to_peer = self.lookup_peer(cluster_id, to_peer_id).await?.context( + error::PeerUnavailableSnafu { + peer_id: to_peer_id, + }, + )?; + let procedure_id = self + .region_migration_manager + .submit_procedure(RegionMigrationProcedureTask { + cluster_id, + region_id, + from_peer, + to_peer, + }) + .await?; + + Ok(SubmitRegionMigrationTaskResponse { + procedure_id: procedure_id.map(|id| id.to_string()), + }) } } diff --git a/src/meta-srv/src/service/heartbeat.rs b/src/meta-srv/src/service/heartbeat.rs index 4144bc30605d..598edf2ca765 100644 --- a/src/meta-srv/src/service/heartbeat.rs +++ b/src/meta-srv/src/service/heartbeat.rs @@ -113,7 +113,7 @@ impl heartbeat_server::Heartbeat for MetaSrv { ); if let Some(key) = pusher_key { - let _ = handler_group.unregister(&key).await; + let _ = handler_group.deregister(&key).await; } }); From 301ffc1d911c349f9f9086bae45bbe7a985a0539 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Fri, 29 Dec 2023 16:46:48 +0900 Subject: [PATCH 04/21] feat(remote_wal): append a noop record after kafka topic initialization (#3040) * feat: append a noop record after kafka topic initialization * chore: apply suggestions from CR * feat: ignore the noop record during the read --- src/common/meta/src/error.rs | 23 +++++++++ .../meta/src/wal/kafka/topic_manager.rs | 48 +++++++++++++++++-- src/log-store/src/kafka/log_store.rs | 4 ++ 3 files changed, 70 insertions(+), 5 deletions(-) diff --git a/src/common/meta/src/error.rs b/src/common/meta/src/error.rs index c120c8ba939d..323d922b9cda 100644 --- a/src/common/meta/src/error.rs +++ b/src/common/meta/src/error.rs @@ -321,6 +321,27 @@ pub enum Error { error: rskafka::client::error::Error, }, + #[snafu(display( + "Failed to build a Kafka partition client, topic: {}, partition: {}", + topic, + partition + ))] + BuildKafkaPartitionClient { + topic: String, + partition: i32, + location: Location, + #[snafu(source)] + error: rskafka::client::error::Error, + }, + + #[snafu(display("Failed to produce records to Kafka, topic: {}", topic))] + ProduceRecord { + topic: String, + location: Location, + #[snafu(source)] + error: rskafka::client::error::Error, + }, + #[snafu(display("Failed to create a Kafka wal topic"))] CreateKafkaWalTopic { location: Location, @@ -368,6 +389,8 @@ impl ErrorExt for Error { | EncodeWalOptions { .. } | BuildKafkaClient { .. } | BuildKafkaCtrlClient { .. } + | BuildKafkaPartitionClient { .. } + | ProduceRecord { .. } | CreateKafkaWalTopic { .. } | EmptyTopicPool { .. } => StatusCode::Unexpected, diff --git a/src/common/meta/src/wal/kafka/topic_manager.rs b/src/common/meta/src/wal/kafka/topic_manager.rs index 860192b97071..80aaa90d402f 100644 --- a/src/common/meta/src/wal/kafka/topic_manager.rs +++ b/src/common/meta/src/wal/kafka/topic_manager.rs @@ -21,13 +21,16 @@ use common_telemetry::{debug, error, info}; use rskafka::client::controller::ControllerClient; use rskafka::client::error::Error as RsKafkaError; use rskafka::client::error::ProtocolError::TopicAlreadyExists; -use rskafka::client::ClientBuilder; +use rskafka::client::partition::{Compression, UnknownTopicHandling}; +use rskafka::client::{Client, ClientBuilder}; +use rskafka::record::Record; use rskafka::BackoffConfig; use snafu::{ensure, AsErrorSource, ResultExt}; use crate::error::{ - BuildKafkaClientSnafu, BuildKafkaCtrlClientSnafu, CreateKafkaWalTopicSnafu, DecodeJsonSnafu, - EncodeJsonSnafu, InvalidNumTopicsSnafu, Result, + BuildKafkaClientSnafu, BuildKafkaCtrlClientSnafu, BuildKafkaPartitionClientSnafu, + CreateKafkaWalTopicSnafu, DecodeJsonSnafu, EncodeJsonSnafu, InvalidNumTopicsSnafu, + ProduceRecordSnafu, Result, }; use crate::kv_backend::KvBackendRef; use crate::rpc::store::PutRequest; @@ -37,6 +40,10 @@ use crate::wal::kafka::KafkaConfig; const CREATED_TOPICS_KEY: &str = "__created_wal_topics/kafka/"; +// Each topic only has one partition for now. +// The `DEFAULT_PARTITION` refers to the index of the partition. +const DEFAULT_PARTITION: i32 = 0; + /// Manages topic initialization and selection. pub struct TopicManager { config: KafkaConfig, @@ -117,14 +124,20 @@ impl TopicManager { .await .with_context(|_| BuildKafkaClientSnafu { broker_endpoints: self.config.broker_endpoints.clone(), - })? + })?; + + let control_client = client .controller_client() .context(BuildKafkaCtrlClientSnafu)?; // Try to create missing topics. let tasks = to_be_created .iter() - .map(|i| self.try_create_topic(&topics[*i], &client)) + .map(|i| async { + self.try_create_topic(&topics[*i], &control_client).await?; + self.try_append_noop_record(&topics[*i], &client).await?; + Ok(()) + }) .collect::>(); futures::future::try_join_all(tasks).await.map(|_| ()) } @@ -141,6 +154,31 @@ impl TopicManager { .collect() } + async fn try_append_noop_record(&self, topic: &Topic, client: &Client) -> Result<()> { + let partition_client = client + .partition_client(topic, DEFAULT_PARTITION, UnknownTopicHandling::Retry) + .await + .context(BuildKafkaPartitionClientSnafu { + topic, + partition: DEFAULT_PARTITION, + })?; + + partition_client + .produce( + vec![Record { + key: None, + value: None, + timestamp: rskafka::chrono::Utc::now(), + headers: Default::default(), + }], + Compression::NoCompression, + ) + .await + .context(ProduceRecordSnafu { topic })?; + + Ok(()) + } + async fn try_create_topic(&self, topic: &Topic, client: &ControllerClient) -> Result<()> { match client .create_topic( diff --git a/src/log-store/src/kafka/log_store.rs b/src/log-store/src/kafka/log_store.rs index 73b0fe1de2a9..df64fa66571f 100644 --- a/src/log-store/src/kafka/log_store.rs +++ b/src/log-store/src/kafka/log_store.rs @@ -186,6 +186,10 @@ impl LogStore for KafkaLogStore { record_offset, ns_clone, high_watermark ); + // Ignores the noop record. + if record.record.value.is_none() { + continue; + } let entries = decode_from_record(record.record)?; // Filters entries by region id. From e16f093282831661260ca7ffbf6474f0a1f513dc Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Fri, 29 Dec 2023 17:17:22 +0900 Subject: [PATCH 05/21] test(remote_wal): add sqlness with kafka wal (#3027) * feat(sqlness): add kafka wal config * chore: add sqlness with kafka wal ci config * fix: fix config * chore: apply suggestions from CR * fix: add metasrv config to sqlness with kafka * fix: replay memtable should from flushed_entry_id + 1 * fix: should set append flag to fopen * feat: start wal allocator in standalone meta mode * feat: append a noop record after kafka topic initialization * test: ignore tests temporally * test: change sqlness kafka wal config --- .github/workflows/develop.yml | 31 +++++ Cargo.lock | 1 + src/common/meta/src/wal/kafka.rs | 1 + src/log-store/src/kafka/log_store.rs | 2 - src/meta-srv/src/metasrv.rs | 3 + tests-integration/fixtures/kafka/README.md | 19 +++ .../types/string/scan_big_varchar.result | 112 +++--------------- .../common/types/string/scan_big_varchar.sql | 32 ++--- tests/conf/datanode-test.toml.template | 6 + tests/conf/metasrv-test.toml.template | 10 ++ tests/conf/standalone-test.toml.template | 6 + tests/runner/Cargo.toml | 1 + tests/runner/src/env.rs | 39 +++++- tests/runner/src/main.rs | 35 +++++- 14 files changed, 176 insertions(+), 122 deletions(-) create mode 100644 tests-integration/fixtures/kafka/README.md create mode 100644 tests/conf/metasrv-test.toml.template diff --git a/.github/workflows/develop.yml b/.github/workflows/develop.yml index f345f0607077..f58e9546a903 100644 --- a/.github/workflows/develop.yml +++ b/.github/workflows/develop.yml @@ -104,6 +104,37 @@ jobs: path: ${{ runner.temp }}/greptime-*.log retention-days: 3 + sqlness-kafka-wal: + name: Sqlness Test with Kafka Wal + if: github.event.pull_request.draft == false + runs-on: ${{ matrix.os }} + strategy: + matrix: + os: [ ubuntu-20.04-8-cores ] + timeout-minutes: 60 + steps: + - uses: actions/checkout@v3 + - uses: arduino/setup-protoc@v1 + with: + repo-token: ${{ secrets.GITHUB_TOKEN }} + - uses: dtolnay/rust-toolchain@master + with: + toolchain: ${{ env.RUST_TOOLCHAIN }} + - name: Rust Cache + uses: Swatinem/rust-cache@v2 + - name: Setup kafka server + working-directory: tests-integration/fixtures/kafka + run: docker compose -f docker-compose-standalone.yml up -d --wait + - name: Run sqlness + run: cargo sqlness -w kafka -k 127.0.0.1:9092 + - name: Upload sqlness logs + if: always() + uses: actions/upload-artifact@v3 + with: + name: sqlness-logs + path: ${{ runner.temp }}/greptime-*.log + retention-days: 3 + fmt: name: Rustfmt if: github.event.pull_request.draft == false diff --git a/Cargo.lock b/Cargo.lock index abe0acb61213..d179ea6c8c54 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8859,6 +8859,7 @@ dependencies = [ "common-recordbatch", "common-time", "serde", + "serde_json", "sqlness", "tinytemplate", "tokio", diff --git a/src/common/meta/src/wal/kafka.rs b/src/common/meta/src/wal/kafka.rs index 0a61b6015dfc..6719f2f63849 100644 --- a/src/common/meta/src/wal/kafka.rs +++ b/src/common/meta/src/wal/kafka.rs @@ -27,6 +27,7 @@ pub use crate::wal::kafka::topic_manager::TopicManager; /// Configurations for kafka wal. #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +#[serde(default)] pub struct KafkaConfig { /// The broker endpoints of the Kafka cluster. pub broker_endpoints: Vec, diff --git a/src/log-store/src/kafka/log_store.rs b/src/log-store/src/kafka/log_store.rs index df64fa66571f..36c86987041b 100644 --- a/src/log-store/src/kafka/log_store.rs +++ b/src/log-store/src/kafka/log_store.rs @@ -197,8 +197,6 @@ impl LogStore for KafkaLogStore { && entry.ns.region_id == region_id { yield Ok(entries); - } else { - yield Ok(vec![]); } // Terminates the stream if the entry with the end offset was read. diff --git a/src/meta-srv/src/metasrv.rs b/src/meta-srv/src/metasrv.rs index dba3c4485002..c5b11874d1c4 100644 --- a/src/meta-srv/src/metasrv.rs +++ b/src/meta-srv/src/metasrv.rs @@ -330,6 +330,9 @@ impl MetaSrv { info!("MetaSrv stopped"); }); } else { + if let Err(e) = self.wal_options_allocator.start().await { + error!(e; "Failed to start wal options allocator"); + } // Always load kv into cached kv store. self.leader_cached_kv_backend .load() diff --git a/tests-integration/fixtures/kafka/README.md b/tests-integration/fixtures/kafka/README.md new file mode 100644 index 000000000000..9d49a2289309 --- /dev/null +++ b/tests-integration/fixtures/kafka/README.md @@ -0,0 +1,19 @@ +## Starts a standalone kafka +```bash +docker compose -f docker-compose-standalone.yml up kafka -d +``` + +## Lists running services +```bash +docker compose -f docker-compose-standalone.yml ps +``` + +## Stops the standalone kafka +```bash +docker compose -f docker-compose-standalone.yml stop kafka +``` + +## Stops and removes the standalone kafka +```bash +docker compose -f docker-compose-standalone.yml down kafka +``` \ No newline at end of file diff --git a/tests/cases/standalone/common/types/string/scan_big_varchar.result b/tests/cases/standalone/common/types/string/scan_big_varchar.result index d132adce3f29..5a14cc0e1996 100644 --- a/tests/cases/standalone/common/types/string/scan_big_varchar.result +++ b/tests/cases/standalone/common/types/string/scan_big_varchar.result @@ -126,102 +126,22 @@ SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; | 128 | 128 | 10000 | 1280000 | +----------+-------------------+-----------------------------------+-----------------------------------+ -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 53 FROM bigtable; - -Affected Rows: 128 - -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; - -+----------+-------------------+-----------------------------------+-----------------------------------+ -| COUNT(*) | COUNT(bigtable.a) | MAX(character_length(bigtable.a)) | SUM(character_length(bigtable.a)) | -+----------+-------------------+-----------------------------------+-----------------------------------+ -| 256 | 256 | 10000 | 2560000 | -+----------+-------------------+-----------------------------------+-----------------------------------+ - -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 57 FROM bigtable; - -Affected Rows: 256 - -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; - -+----------+-------------------+-----------------------------------+-----------------------------------+ -| COUNT(*) | COUNT(bigtable.a) | MAX(character_length(bigtable.a)) | SUM(character_length(bigtable.a)) | -+----------+-------------------+-----------------------------------+-----------------------------------+ -| 512 | 512 | 10000 | 5120000 | -+----------+-------------------+-----------------------------------+-----------------------------------+ - -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 61 FROM bigtable; - -Affected Rows: 512 - -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; - -+----------+-------------------+-----------------------------------+-----------------------------------+ -| COUNT(*) | COUNT(bigtable.a) | MAX(character_length(bigtable.a)) | SUM(character_length(bigtable.a)) | -+----------+-------------------+-----------------------------------+-----------------------------------+ -| 1024 | 1024 | 10000 | 10240000 | -+----------+-------------------+-----------------------------------+-----------------------------------+ - -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 63 FROM bigtable; - -Affected Rows: 1024 - -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; - -+----------+-------------------+-----------------------------------+-----------------------------------+ -| COUNT(*) | COUNT(bigtable.a) | MAX(character_length(bigtable.a)) | SUM(character_length(bigtable.a)) | -+----------+-------------------+-----------------------------------+-----------------------------------+ -| 2048 | 2048 | 10000 | 20480000 | -+----------+-------------------+-----------------------------------+-----------------------------------+ - -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 67 FROM bigtable; - -Affected Rows: 2048 - -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; - -+----------+-------------------+-----------------------------------+-----------------------------------+ -| COUNT(*) | COUNT(bigtable.a) | MAX(character_length(bigtable.a)) | SUM(character_length(bigtable.a)) | -+----------+-------------------+-----------------------------------+-----------------------------------+ -| 4096 | 4096 | 10000 | 40960000 | -+----------+-------------------+-----------------------------------+-----------------------------------+ - -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 71 FROM bigtable; - -Affected Rows: 4096 - -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; - -+----------+-------------------+-----------------------------------+-----------------------------------+ -| COUNT(*) | COUNT(bigtable.a) | MAX(character_length(bigtable.a)) | SUM(character_length(bigtable.a)) | -+----------+-------------------+-----------------------------------+-----------------------------------+ -| 8192 | 8192 | 10000 | 81920000 | -+----------+-------------------+-----------------------------------+-----------------------------------+ - -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 73 FROM bigtable; - -Affected Rows: 8192 - -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; - -+----------+-------------------+-----------------------------------+-----------------------------------+ -| COUNT(*) | COUNT(bigtable.a) | MAX(character_length(bigtable.a)) | SUM(character_length(bigtable.a)) | -+----------+-------------------+-----------------------------------+-----------------------------------+ -| 16384 | 16384 | 10000 | 163840000 | -+----------+-------------------+-----------------------------------+-----------------------------------+ - -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 79 FROM bigtable; - -Affected Rows: 16384 - -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; - -+----------+-------------------+-----------------------------------+-----------------------------------+ -| COUNT(*) | COUNT(bigtable.a) | MAX(character_length(bigtable.a)) | SUM(character_length(bigtable.a)) | -+----------+-------------------+-----------------------------------+-----------------------------------+ -| 32768 | 32768 | 10000 | 327680000 | -+----------+-------------------+-----------------------------------+-----------------------------------+ - +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 53 FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 57 FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 61 FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 63 FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 67 FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 71 FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 73 FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 79 FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; DROP TABLE test; Affected Rows: 0 diff --git a/tests/cases/standalone/common/types/string/scan_big_varchar.sql b/tests/cases/standalone/common/types/string/scan_big_varchar.sql index 81bcb19da9ed..d9ce27e041f8 100644 --- a/tests/cases/standalone/common/types/string/scan_big_varchar.sql +++ b/tests/cases/standalone/common/types/string/scan_big_varchar.sql @@ -51,38 +51,38 @@ INSERT INTO bigtable SELECT a, to_unixtime(ts) * 51 FROM bigtable; SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 53 FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 53 FROM bigtable; -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 57 FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 57 FROM bigtable; -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 61 FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 61 FROM bigtable; -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 63 FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 63 FROM bigtable; -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 67 FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 67 FROM bigtable; -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 71 FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 71 FROM bigtable; -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 73 FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 73 FROM bigtable; -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; -INSERT INTO bigtable SELECT a, to_unixtime(ts) * 79 FROM bigtable; +-- INSERT INTO bigtable SELECT a, to_unixtime(ts) * 79 FROM bigtable; -SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; +-- SELECT COUNT(*), COUNT(a), MAX(LENGTH(a)), SUM(LENGTH(a)) FROM bigtable; DROP TABLE test; diff --git a/tests/conf/datanode-test.toml.template b/tests/conf/datanode-test.toml.template index 55d0c2f1fe4c..04968bc95690 100644 --- a/tests/conf/datanode-test.toml.template +++ b/tests/conf/datanode-test.toml.template @@ -6,12 +6,18 @@ rpc_hostname = '127.0.0.1' rpc_runtime_size = 8 [wal] +{{ if is_raft_engine }} provider = "raft_engine" file_size = '1GB' purge_interval = '10m' purge_threshold = '10GB' read_batch_size = 128 sync_write = false +{{ else }} +provider = "kafka" +broker_endpoints = {kafka_wal_broker_endpoints | unescaped} +linger = "5ms" +{{ endif }} [storage] type = 'File' diff --git a/tests/conf/metasrv-test.toml.template b/tests/conf/metasrv-test.toml.template new file mode 100644 index 000000000000..ecd69473249a --- /dev/null +++ b/tests/conf/metasrv-test.toml.template @@ -0,0 +1,10 @@ +[wal] +{{ if is_raft_engine }} +provider = "raft_engine" +{{ else }} +provider = "kafka" +broker_endpoints = {kafka_wal_broker_endpoints | unescaped} +num_topics = 64 +selector_type = "round_robin" +topic_name_prefix = "distributed_test_greptimedb_wal_topic" +{{ endif }} diff --git a/tests/conf/standalone-test.toml.template b/tests/conf/standalone-test.toml.template index f0ddc38d048e..2e30ac35c266 100644 --- a/tests/conf/standalone-test.toml.template +++ b/tests/conf/standalone-test.toml.template @@ -3,12 +3,18 @@ enable_memory_catalog = false require_lease_before_startup = true [wal] +{{ if is_raft_engine }} provider = "raft_engine" file_size = '1GB' purge_interval = '10m' purge_threshold = '10GB' read_batch_size = 128 sync_write = false +{{ else }} +provider = "kafka" +broker_endpoints = {kafka_wal_broker_endpoints | unescaped} +linger = "5ms" +{{ endif }} [storage] type = 'File' diff --git a/tests/runner/Cargo.toml b/tests/runner/Cargo.toml index 7b9141776fd8..b2757f479dd6 100644 --- a/tests/runner/Cargo.toml +++ b/tests/runner/Cargo.toml @@ -15,6 +15,7 @@ common-query.workspace = true common-recordbatch.workspace = true common-time.workspace = true serde.workspace = true +serde_json.workspace = true sqlness = { version = "0.5" } tinytemplate = "1.2" tokio.workspace = true diff --git a/tests/runner/src/env.rs b/tests/runner/src/env.rs index 1bd7ad36496a..76946ad6ba60 100644 --- a/tests/runner/src/env.rs +++ b/tests/runner/src/env.rs @@ -41,10 +41,17 @@ const METASRV_ADDR: &str = "127.0.0.1:3002"; const SERVER_ADDR: &str = "127.0.0.1:4001"; const DEFAULT_LOG_LEVEL: &str = "--log-level=debug,hyper=warn,tower=warn,datafusion=warn,reqwest=warn,sqlparser=warn,h2=info,opendal=info"; +#[derive(Clone)] +pub enum WalConfig { + RaftEngine, + Kafka { broker_endpoints: Vec }, +} + #[derive(Clone)] pub struct Env { data_home: PathBuf, server_addr: Option, + wal: WalConfig, } #[allow(clippy::print_stdout)] @@ -68,10 +75,11 @@ impl EnvController for Env { #[allow(clippy::print_stdout)] impl Env { - pub fn new(data_home: PathBuf, server_addr: Option) -> Self { + pub fn new(data_home: PathBuf, server_addr: Option, wal: WalConfig) -> Self { Self { data_home, server_addr, + wal, } } @@ -81,7 +89,7 @@ impl Env { } else { Self::build_db().await; - let db_ctx = GreptimeDBContext::new(); + let db_ctx = GreptimeDBContext::new(self.wal.clone()); let server_process = self.start_server("standalone", &db_ctx, true).await; @@ -106,7 +114,7 @@ impl Env { } else { Self::build_db().await; - let db_ctx = GreptimeDBContext::new(); + let db_ctx = GreptimeDBContext::new(self.wal.clone()); // start a distributed GreptimeDB let meta_server = self.start_server("metasrv", &db_ctx, true).await; @@ -145,6 +153,7 @@ impl Env { ctx: GreptimeDBContext { time: 0, datanode_id: Default::default(), + wal: self.wal.clone(), }, is_standalone: false, env: self.clone(), @@ -178,6 +187,7 @@ impl Env { .create(true) .write(true) .truncate(truncate_log) + .append(!truncate_log) .open(log_file_name) .unwrap(); @@ -214,6 +224,8 @@ impl Env { "--enable-region-failover".to_string(), "false".to_string(), "--http-addr=127.0.0.1:5002".to_string(), + "-c".to_string(), + self.generate_config_file(subcommand, db_ctx), ]; (args, METASRV_ADDR.to_string()) } @@ -321,6 +333,8 @@ impl Env { wal_dir: String, data_home: String, procedure_dir: String, + is_raft_engine: bool, + kafka_wal_broker_endpoints: String, } let data_home = self @@ -334,6 +348,8 @@ impl Env { wal_dir, data_home: data_home.display().to_string(), procedure_dir, + is_raft_engine: db_ctx.is_raft_engine(), + kafka_wal_broker_endpoints: db_ctx.kafka_wal_broker_endpoints(), }; let rendered = tt.render(subcommand, &ctx).unwrap(); @@ -447,13 +463,28 @@ struct GreptimeDBContext { /// Start time in millisecond time: i64, datanode_id: AtomicU32, + wal: WalConfig, } impl GreptimeDBContext { - pub fn new() -> Self { + pub fn new(wal: WalConfig) -> Self { Self { time: common_time::util::current_time_millis(), datanode_id: AtomicU32::new(0), + wal, + } + } + + fn is_raft_engine(&self) -> bool { + matches!(self.wal, WalConfig::RaftEngine) + } + + fn kafka_wal_broker_endpoints(&self) -> String { + match &self.wal { + WalConfig::RaftEngine => String::new(), + WalConfig::Kafka { broker_endpoints } => { + serde_json::to_string(&broker_endpoints).unwrap() + } } } diff --git a/tests/runner/src/main.rs b/tests/runner/src/main.rs index 9c93e628a484..5fdddbd7149a 100644 --- a/tests/runner/src/main.rs +++ b/tests/runner/src/main.rs @@ -14,13 +14,20 @@ use std::path::PathBuf; -use clap::Parser; -use env::Env; +use clap::{Parser, ValueEnum}; +use env::{Env, WalConfig}; use sqlness::{ConfigBuilder, Runner}; mod env; mod util; +#[derive(ValueEnum, Debug, Clone)] +#[clap(rename_all = "snake_case")] +enum Wal { + RaftEngine, + Kafka, +} + #[derive(Parser, Debug)] #[clap(author, version, about, long_about = None)] /// SQL Harness for GrepTimeDB @@ -41,9 +48,17 @@ struct Args { #[clap(short, long, default_value = ".*")] test_filter: String, - /// Address of the server + /// Address of the server. #[clap(short, long)] server_addr: Option, + + /// The type of Wal. + #[clap(short, long, default_value = "raft_engine")] + wal: Wal, + + /// The kafka wal broker endpoints. + #[clap(short, long, default_value = "127.0.0.1:9092")] + kafka_wal_broker_endpoints: String, } #[tokio::main] @@ -63,6 +78,18 @@ async fn main() { .env_config_file(args.env_config_file) .build() .unwrap(); - let runner = Runner::new(config, Env::new(data_home, args.server_addr)); + + let wal = match args.wal { + Wal::RaftEngine => WalConfig::RaftEngine, + Wal::Kafka => WalConfig::Kafka { + broker_endpoints: args + .kafka_wal_broker_endpoints + .split(',') + .map(|s| s.trim().to_string()) + .collect(), + }, + }; + + let runner = Runner::new(config, Env::new(data_home, args.server_addr, wal)); runner.run().await.unwrap(); } From 7551432cff623c18c41411e693785bdfc94f8845 Mon Sep 17 00:00:00 2001 From: LFC <990479+MichaelScofield@users.noreply.github.com> Date: Fri, 29 Dec 2023 16:50:59 +0800 Subject: [PATCH 06/21] refactor: merge standalone and metasrv table metadata allocators (#3035) * refactor: merge standalone and metasrv table metadata allocators * Update src/common/meta/src/ddl/table_meta.rs Co-authored-by: niebayes * Update src/common/meta/src/ddl/table_meta.rs Co-authored-by: Weny Xu --------- Co-authored-by: niebayes Co-authored-by: Weny Xu --- src/cmd/src/standalone.rs | 12 +- src/common/meta/src/ddl.rs | 14 +- src/common/meta/src/ddl/table_meta.rs | 190 ++++++++++++++++++ src/common/meta/src/ddl_manager.rs | 33 +-- src/frontend/src/instance/standalone.rs | 136 +------------ src/meta-srv/src/metasrv.rs | 2 +- src/meta-srv/src/metasrv/builder.rs | 19 +- src/meta-srv/src/procedure/region_failover.rs | 3 +- .../region_failover/failover_start.rs | 2 +- src/meta-srv/src/selector/common.rs | 4 +- src/meta-srv/src/selector/lease_based.rs | 2 +- src/meta-srv/src/selector/load_based.rs | 2 +- src/meta-srv/src/selector/weight_compute.rs | 4 +- src/meta-srv/src/table_meta_alloc.rs | 168 ++++------------ tests-integration/src/standalone.rs | 8 +- tests-integration/tests/region_failover.rs | 2 +- 16 files changed, 279 insertions(+), 322 deletions(-) create mode 100644 src/common/meta/src/ddl/table_meta.rs diff --git a/src/cmd/src/standalone.rs b/src/cmd/src/standalone.rs index 27cc19fd6655..0ce6e72bb6f2 100644 --- a/src/cmd/src/standalone.rs +++ b/src/cmd/src/standalone.rs @@ -22,7 +22,8 @@ use common_config::wal::StandaloneWalConfig; use common_config::{metadata_store_dir, KvBackendConfig}; use common_meta::cache_invalidator::DummyCacheInvalidator; use common_meta::datanode_manager::DatanodeManagerRef; -use common_meta::ddl::{DdlTaskExecutorRef, TableMetadataAllocatorRef}; +use common_meta::ddl::table_meta::TableMetadataAllocator; +use common_meta::ddl::DdlTaskExecutorRef; use common_meta::ddl_manager::DdlManager; use common_meta::key::{TableMetadataManager, TableMetadataManagerRef}; use common_meta::kv_backend::KvBackendRef; @@ -38,7 +39,6 @@ use datanode::datanode::{Datanode, DatanodeBuilder}; use file_engine::config::EngineConfig as FileEngineConfig; use frontend::frontend::FrontendOptions; use frontend::instance::builder::FrontendBuilder; -use frontend::instance::standalone::StandaloneTableMetadataAllocator; use frontend::instance::{FrontendInstance, Instance as FeInstance, StandaloneDatanodeManager}; use frontend::service_config::{ GrpcOptions, InfluxdbOptions, MysqlOptions, OpentsdbOptions, PostgresOptions, PromStoreOptions, @@ -406,10 +406,8 @@ impl StartCommand { opts.wal_meta.clone(), kv_backend.clone(), )); - let table_meta_allocator = Arc::new(StandaloneTableMetadataAllocator::new( - table_id_sequence, - wal_options_allocator.clone(), - )); + let table_meta_allocator = + TableMetadataAllocator::new(table_id_sequence, wal_options_allocator.clone()); let ddl_task_executor = Self::create_ddl_task_executor( kv_backend.clone(), @@ -446,7 +444,7 @@ impl StartCommand { kv_backend: KvBackendRef, procedure_manager: ProcedureManagerRef, datanode_manager: DatanodeManagerRef, - table_meta_allocator: TableMetadataAllocatorRef, + table_meta_allocator: TableMetadataAllocator, ) -> Result { let table_metadata_manager = Self::create_table_metadata_manager(kv_backend.clone()).await?; diff --git a/src/common/meta/src/ddl.rs b/src/common/meta/src/ddl.rs index bb5220724ab6..05e076f8ae47 100644 --- a/src/common/meta/src/ddl.rs +++ b/src/common/meta/src/ddl.rs @@ -24,11 +24,12 @@ use crate::error::Result; use crate::key::table_route::TableRouteValue; use crate::key::TableMetadataManagerRef; use crate::region_keeper::MemoryRegionKeeperRef; -use crate::rpc::ddl::{CreateTableTask, SubmitDdlTaskRequest, SubmitDdlTaskResponse}; +use crate::rpc::ddl::{SubmitDdlTaskRequest, SubmitDdlTaskResponse}; pub mod alter_table; pub mod create_table; pub mod drop_table; +pub mod table_meta; pub mod truncate_table; pub mod utils; @@ -64,17 +65,6 @@ pub struct TableMetadata { pub region_wal_options: HashMap, } -#[async_trait::async_trait] -pub trait TableMetadataAllocator: Send + Sync { - async fn create( - &self, - ctx: &TableMetadataAllocatorContext, - task: &CreateTableTask, - ) -> Result; -} - -pub type TableMetadataAllocatorRef = Arc; - #[derive(Clone)] pub struct DdlContext { pub datanode_manager: DatanodeManagerRef, diff --git a/src/common/meta/src/ddl/table_meta.rs b/src/common/meta/src/ddl/table_meta.rs new file mode 100644 index 000000000000..8bd460d98a74 --- /dev/null +++ b/src/common/meta/src/ddl/table_meta.rs @@ -0,0 +1,190 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::Arc; + +use async_trait::async_trait; +use common_catalog::consts::METRIC_ENGINE; +use common_telemetry::{debug, info}; +use snafu::ensure; +use store_api::storage::{RegionId, RegionNumber, TableId}; + +use crate::ddl::{TableMetadata, TableMetadataAllocatorContext}; +use crate::error::{Result, UnsupportedSnafu}; +use crate::key::table_route::{LogicalTableRouteValue, PhysicalTableRouteValue, TableRouteValue}; +use crate::peer::Peer; +use crate::rpc::ddl::CreateTableTask; +use crate::rpc::router::{Region, RegionRoute}; +use crate::sequence::SequenceRef; +use crate::wal::{allocate_region_wal_options, WalOptionsAllocatorRef}; + +pub struct TableMetadataAllocator { + table_id_sequence: SequenceRef, + wal_options_allocator: WalOptionsAllocatorRef, + peer_allocator: PeerAllocatorRef, +} + +impl TableMetadataAllocator { + pub fn new( + table_id_sequence: SequenceRef, + wal_options_allocator: WalOptionsAllocatorRef, + ) -> Self { + Self::with_peer_allocator( + table_id_sequence, + wal_options_allocator, + Arc::new(NoopPeerAllocator), + ) + } + + pub fn with_peer_allocator( + table_id_sequence: SequenceRef, + wal_options_allocator: WalOptionsAllocatorRef, + peer_allocator: PeerAllocatorRef, + ) -> Self { + Self { + table_id_sequence, + wal_options_allocator, + peer_allocator, + } + } + + async fn allocate_table_id(&self, task: &CreateTableTask) -> Result { + let table_id = if let Some(table_id) = &task.create_table.table_id { + let table_id = table_id.id; + + ensure!( + !self + .table_id_sequence + .min_max() + .await + .contains(&(table_id as u64)), + UnsupportedSnafu { + operation: format!( + "create table by id {} that is reserved in this node", + table_id + ) + } + ); + + info!( + "Received explicitly allocated table id {}, will use it directly.", + table_id + ); + + table_id + } else { + self.table_id_sequence.next().await? as TableId + }; + Ok(table_id) + } + + fn create_wal_options( + &self, + table_route: &TableRouteValue, + ) -> Result> { + match table_route { + TableRouteValue::Physical(x) => { + let region_numbers = x + .region_routes + .iter() + .map(|route| route.region.id.region_number()) + .collect(); + allocate_region_wal_options(region_numbers, &self.wal_options_allocator) + } + TableRouteValue::Logical(_) => Ok(HashMap::new()), + } + } + + async fn create_table_route( + &self, + ctx: &TableMetadataAllocatorContext, + table_id: TableId, + task: &CreateTableTask, + ) -> Result { + let regions = task.partitions.len(); + + let table_route = if task.create_table.engine == METRIC_ENGINE { + TableRouteValue::Logical(LogicalTableRouteValue {}) + } else { + let peers = self.peer_allocator.alloc(ctx, regions).await?; + + let region_routes = task + .partitions + .iter() + .enumerate() + .map(|(i, partition)| { + let region = Region { + id: RegionId::new(table_id, i as u32), + partition: Some(partition.clone().into()), + ..Default::default() + }; + + let peer = peers[i % peers.len()].clone(); + + RegionRoute { + region, + leader_peer: Some(peer), + ..Default::default() + } + }) + .collect::>(); + TableRouteValue::Physical(PhysicalTableRouteValue::new(region_routes)) + }; + Ok(table_route) + } + pub async fn create( + &self, + ctx: &TableMetadataAllocatorContext, + task: &CreateTableTask, + ) -> Result { + let table_id = self.allocate_table_id(task).await?; + let table_route = self.create_table_route(ctx, table_id, task).await?; + let region_wal_options = self.create_wal_options(&table_route)?; + + debug!( + "Allocated region wal options {:?} for table {}", + region_wal_options, table_id + ); + + Ok(TableMetadata { + table_id, + table_route, + region_wal_options, + }) + } +} + +pub type PeerAllocatorRef = Arc; + +/// [PeerAllocator] allocates [Peer]s for creating regions. +#[async_trait] +pub trait PeerAllocator: Send + Sync { + /// Allocates `regions` size [Peer]s. + async fn alloc(&self, ctx: &TableMetadataAllocatorContext, regions: usize) + -> Result>; +} + +struct NoopPeerAllocator; + +#[async_trait] +impl PeerAllocator for NoopPeerAllocator { + async fn alloc( + &self, + _ctx: &TableMetadataAllocatorContext, + regions: usize, + ) -> Result> { + Ok(vec![Peer::default(); regions]) + } +} diff --git a/src/common/meta/src/ddl_manager.rs b/src/common/meta/src/ddl_manager.rs index 6b1e4bf94f38..af669797f4d4 100644 --- a/src/common/meta/src/ddl_manager.rs +++ b/src/common/meta/src/ddl_manager.rs @@ -26,10 +26,10 @@ use crate::datanode_manager::DatanodeManagerRef; use crate::ddl::alter_table::AlterTableProcedure; use crate::ddl::create_table::CreateTableProcedure; use crate::ddl::drop_table::DropTableProcedure; +use crate::ddl::table_meta::TableMetadataAllocator; use crate::ddl::truncate_table::TruncateTableProcedure; use crate::ddl::{ DdlContext, DdlTaskExecutor, ExecutorContext, TableMetadata, TableMetadataAllocatorContext, - TableMetadataAllocatorRef, }; use crate::error::{ self, RegisterProcedureLoaderSnafu, Result, SubmitProcedureSnafu, TableNotFoundSnafu, @@ -54,7 +54,7 @@ pub struct DdlManager { datanode_manager: DatanodeManagerRef, cache_invalidator: CacheInvalidatorRef, table_metadata_manager: TableMetadataManagerRef, - table_metadata_allocator: TableMetadataAllocatorRef, + table_metadata_allocator: TableMetadataAllocator, memory_region_keeper: MemoryRegionKeeperRef, } @@ -65,7 +65,7 @@ impl DdlManager { datanode_clients: DatanodeManagerRef, cache_invalidator: CacheInvalidatorRef, table_metadata_manager: TableMetadataManagerRef, - table_metadata_allocator: TableMetadataAllocatorRef, + table_metadata_allocator: TableMetadataAllocator, memory_region_keeper: MemoryRegionKeeperRef, ) -> Result { let manager = Self { @@ -461,15 +461,15 @@ mod tests { use crate::ddl::alter_table::AlterTableProcedure; use crate::ddl::create_table::CreateTableProcedure; use crate::ddl::drop_table::DropTableProcedure; + use crate::ddl::table_meta::TableMetadataAllocator; use crate::ddl::truncate_table::TruncateTableProcedure; - use crate::ddl::{TableMetadata, TableMetadataAllocator, TableMetadataAllocatorContext}; - use crate::error::Result; use crate::key::TableMetadataManager; use crate::kv_backend::memory::MemoryKvBackend; use crate::peer::Peer; use crate::region_keeper::MemoryRegionKeeper; - use crate::rpc::ddl::CreateTableTask; + use crate::sequence::SequenceBuilder; use crate::state_store::KvStateStore; + use crate::wal::WalOptionsAllocator; /// A dummy implemented [DatanodeManager]. pub struct DummyDatanodeManager; @@ -481,26 +481,12 @@ mod tests { } } - /// A dummy implemented [TableMetadataAllocator]. - pub struct DummyTableMetadataAllocator; - - #[async_trait::async_trait] - impl TableMetadataAllocator for DummyTableMetadataAllocator { - async fn create( - &self, - _ctx: &TableMetadataAllocatorContext, - _task: &CreateTableTask, - ) -> Result { - unimplemented!() - } - } - #[test] fn test_try_new() { let kv_backend = Arc::new(MemoryKvBackend::new()); let table_metadata_manager = Arc::new(TableMetadataManager::new(kv_backend.clone())); - let state_store = Arc::new(KvStateStore::new(kv_backend)); + let state_store = Arc::new(KvStateStore::new(kv_backend.clone())); let procedure_manager = Arc::new(LocalManager::new(Default::default(), state_store)); let _ = DdlManager::try_new( @@ -508,7 +494,10 @@ mod tests { Arc::new(DummyDatanodeManager), Arc::new(DummyCacheInvalidator), table_metadata_manager, - Arc::new(DummyTableMetadataAllocator), + TableMetadataAllocator::new( + Arc::new(SequenceBuilder::new("test", kv_backend).build()), + Arc::new(WalOptionsAllocator::default()), + ), Arc::new(MemoryRegionKeeper::default()), ); diff --git a/src/frontend/src/instance/standalone.rs b/src/frontend/src/instance/standalone.rs index 21496e28edc5..b7a067d49c16 100644 --- a/src/frontend/src/instance/standalone.rs +++ b/src/frontend/src/instance/standalone.rs @@ -12,33 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; use std::sync::Arc; use api::v1::region::{QueryRequest, RegionRequest, RegionResponse}; use async_trait::async_trait; use client::region::check_response_header; -use common_catalog::consts::METRIC_ENGINE; use common_error::ext::BoxedError; use common_meta::datanode_manager::{AffectedRows, Datanode, DatanodeManager, DatanodeRef}; -use common_meta::ddl::{TableMetadata, TableMetadataAllocator, TableMetadataAllocatorContext}; -use common_meta::error::{self as meta_error, Result as MetaResult, UnsupportedSnafu}; -use common_meta::key::table_route::{ - LogicalTableRouteValue, PhysicalTableRouteValue, TableRouteValue, -}; +use common_meta::error::{self as meta_error, Result as MetaResult}; use common_meta::peer::Peer; -use common_meta::rpc::ddl::CreateTableTask; -use common_meta::rpc::router::{Region, RegionRoute}; -use common_meta::sequence::SequenceRef; -use common_meta::wal::options_allocator::allocate_region_wal_options; -use common_meta::wal::WalOptionsAllocatorRef; use common_recordbatch::SendableRecordBatchStream; +use common_telemetry::tracing; use common_telemetry::tracing_context::{FutureExt, TracingContext}; -use common_telemetry::{debug, info, tracing}; use datanode::region_server::RegionServer; use servers::grpc::region_server::RegionServerHandler; -use snafu::{ensure, OptionExt, ResultExt}; -use store_api::storage::{RegionId, RegionNumber, TableId}; +use snafu::{OptionExt, ResultExt}; use crate::error::{InvalidRegionRequestSnafu, InvokeRegionServerSnafu, Result}; @@ -109,121 +97,3 @@ impl Datanode for RegionInvoker { .context(meta_error::ExternalSnafu) } } - -pub struct StandaloneTableMetadataAllocator { - table_id_sequence: SequenceRef, - wal_options_allocator: WalOptionsAllocatorRef, -} - -impl StandaloneTableMetadataAllocator { - pub fn new( - table_id_sequence: SequenceRef, - wal_options_allocator: WalOptionsAllocatorRef, - ) -> Self { - Self { - table_id_sequence, - wal_options_allocator, - } - } - - async fn allocate_table_id(&self, task: &CreateTableTask) -> MetaResult { - let table_id = if let Some(table_id) = &task.create_table.table_id { - let table_id = table_id.id; - - ensure!( - !self - .table_id_sequence - .min_max() - .await - .contains(&(table_id as u64)), - UnsupportedSnafu { - operation: format!( - "create table by id {} that is reserved in this node", - table_id - ) - } - ); - - info!( - "Received explicitly allocated table id {}, will use it directly.", - table_id - ); - - table_id - } else { - self.table_id_sequence.next().await? as TableId - }; - Ok(table_id) - } - - fn create_wal_options( - &self, - table_route: &TableRouteValue, - ) -> MetaResult> { - match table_route { - TableRouteValue::Physical(x) => { - let region_numbers = x - .region_routes - .iter() - .map(|route| route.region.id.region_number()) - .collect(); - allocate_region_wal_options(region_numbers, &self.wal_options_allocator) - } - TableRouteValue::Logical(_) => Ok(HashMap::new()), - } - } -} - -fn create_table_route(table_id: TableId, task: &CreateTableTask) -> TableRouteValue { - if task.create_table.engine == METRIC_ENGINE { - TableRouteValue::Logical(LogicalTableRouteValue {}) - } else { - let region_routes = task - .partitions - .iter() - .enumerate() - .map(|(i, partition)| { - let region = Region { - id: RegionId::new(table_id, i as u32), - partition: Some(partition.clone().into()), - ..Default::default() - }; - // It's only a placeholder. - let peer = Peer::default(); - RegionRoute { - region, - leader_peer: Some(peer), - follower_peers: vec![], - leader_status: None, - } - }) - .collect::>(); - TableRouteValue::Physical(PhysicalTableRouteValue::new(region_routes)) - } -} - -#[async_trait] -impl TableMetadataAllocator for StandaloneTableMetadataAllocator { - async fn create( - &self, - _ctx: &TableMetadataAllocatorContext, - task: &CreateTableTask, - ) -> MetaResult { - let table_id = self.allocate_table_id(task).await?; - - let table_route = create_table_route(table_id, task); - - let region_wal_options = self.create_wal_options(&table_route)?; - - debug!( - "Allocated region wal options {:?} for table {}", - region_wal_options, table_id - ); - - Ok(TableMetadata { - table_id, - table_route, - region_wal_options, - }) - } -} diff --git a/src/meta-srv/src/metasrv.rs b/src/meta-srv/src/metasrv.rs index c5b11874d1c4..cb0f2cda4965 100644 --- a/src/meta-srv/src/metasrv.rs +++ b/src/meta-srv/src/metasrv.rs @@ -18,13 +18,13 @@ use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use std::time::Duration; -use api::v1::meta::Peer; use common_base::Plugins; use common_greptimedb_telemetry::GreptimeDBTelemetryTask; use common_grpc::channel_manager; use common_meta::ddl::DdlTaskExecutorRef; use common_meta::key::TableMetadataManagerRef; use common_meta::kv_backend::{KvBackendRef, ResettableKvBackend, ResettableKvBackendRef}; +use common_meta::peer::Peer; use common_meta::region_keeper::MemoryRegionKeeperRef; use common_meta::wal::options_allocator::WalOptionsAllocatorRef; use common_meta::wal::WalConfig; diff --git a/src/meta-srv/src/metasrv/builder.rs b/src/meta-srv/src/metasrv/builder.rs index 28a34714d665..a0cd1357d298 100644 --- a/src/meta-srv/src/metasrv/builder.rs +++ b/src/meta-srv/src/metasrv/builder.rs @@ -21,7 +21,7 @@ use common_base::Plugins; use common_catalog::consts::MIN_USER_TABLE_ID; use common_grpc::channel_manager::ChannelConfig; use common_meta::datanode_manager::DatanodeManagerRef; -use common_meta::ddl::TableMetadataAllocatorRef; +use common_meta::ddl::table_meta::TableMetadataAllocator; use common_meta::ddl_manager::{DdlManager, DdlManagerRef}; use common_meta::distributed_time_constants; use common_meta::key::{TableMetadataManager, TableMetadataManagerRef}; @@ -64,7 +64,7 @@ use crate::selector::lease_based::LeaseBasedSelector; use crate::service::mailbox::MailboxRef; use crate::service::store::cached_kv::{CheckLeader, LeaderCachedKvBackend}; use crate::state::State; -use crate::table_meta_alloc::MetaSrvTableMetadataAllocator; +use crate::table_meta_alloc::MetasrvPeerAllocator; // TODO(fys): try use derive_builder macro pub struct MetaSrvBuilder { @@ -78,7 +78,7 @@ pub struct MetaSrvBuilder { lock: Option, datanode_manager: Option, plugins: Option, - table_metadata_allocator: Option, + table_metadata_allocator: Option, } impl MetaSrvBuilder { @@ -150,7 +150,7 @@ impl MetaSrvBuilder { pub fn table_metadata_allocator( mut self, - table_metadata_allocator: TableMetadataAllocatorRef, + table_metadata_allocator: TableMetadataAllocator, ) -> Self { self.table_metadata_allocator = Some(table_metadata_allocator); self @@ -218,12 +218,15 @@ impl MetaSrvBuilder { .step(10) .build(), ); - Arc::new(MetaSrvTableMetadataAllocator::new( + let peer_allocator = Arc::new(MetasrvPeerAllocator::new( selector_ctx.clone(), selector.clone(), - sequence.clone(), + )); + TableMetadataAllocator::with_peer_allocator( + sequence, wal_options_allocator.clone(), - )) + peer_allocator, + ) }); let opening_region_keeper = Arc::new(MemoryRegionKeeper::default()); @@ -382,7 +385,7 @@ fn build_ddl_manager( procedure_manager: &ProcedureManagerRef, mailbox: &MailboxRef, table_metadata_manager: &TableMetadataManagerRef, - table_metadata_allocator: TableMetadataAllocatorRef, + table_metadata_allocator: TableMetadataAllocator, memory_region_keeper: &MemoryRegionKeeperRef, ) -> Result { let datanode_clients = datanode_clients.unwrap_or_else(|| { diff --git a/src/meta-srv/src/procedure/region_failover.rs b/src/meta-srv/src/procedure/region_failover.rs index 7566ce7f3fec..50ab0e742307 100644 --- a/src/meta-srv/src/procedure/region_failover.rs +++ b/src/meta-srv/src/procedure/region_failover.rs @@ -383,12 +383,13 @@ mod tests { use std::sync::Mutex; use api::v1::meta::mailbox_message::Payload; - use api::v1::meta::{HeartbeatResponse, MailboxMessage, Peer, RequestHeader}; + use api::v1::meta::{HeartbeatResponse, MailboxMessage, RequestHeader}; use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; use common_meta::ddl::utils::region_storage_path; use common_meta::instruction::{Instruction, InstructionReply, OpenRegion, SimpleReply}; use common_meta::key::TableMetadataManager; use common_meta::kv_backend::memory::MemoryKvBackend; + use common_meta::peer::Peer; use common_meta::sequence::SequenceBuilder; use common_meta::DatanodeId; use common_procedure::{BoxedProcedure, ProcedureId}; diff --git a/src/meta-srv/src/procedure/region_failover/failover_start.rs b/src/meta-srv/src/procedure/region_failover/failover_start.rs index d3c2dea8f261..003cd7e4b9c8 100644 --- a/src/meta-srv/src/procedure/region_failover/failover_start.rs +++ b/src/meta-srv/src/procedure/region_failover/failover_start.rs @@ -59,7 +59,7 @@ impl RegionFailoverStart { .iter() .filter_map(|p| { if p.id != failed_region.datanode_id { - Some(p.clone().into()) + Some(p.clone()) } else { None } diff --git a/src/meta-srv/src/selector/common.rs b/src/meta-srv/src/selector/common.rs index 8d807b7eb26c..cccdcd391282 100644 --- a/src/meta-srv/src/selector/common.rs +++ b/src/meta-srv/src/selector/common.rs @@ -14,7 +14,7 @@ use std::collections::HashSet; -use api::v1::meta::Peer; +use common_meta::peer::Peer; use snafu::ensure; use super::weighted_choose::{WeightedChoose, WeightedItem}; @@ -92,7 +92,7 @@ where mod tests { use std::collections::HashSet; - use api::v1::meta::Peer; + use common_meta::peer::Peer; use crate::selector::common::choose_peers; use crate::selector::weighted_choose::{RandomWeightedChoose, WeightedItem}; diff --git a/src/meta-srv/src/selector/lease_based.rs b/src/meta-srv/src/selector/lease_based.rs index 268371a003f7..bdfffacf0529 100644 --- a/src/meta-srv/src/selector/lease_based.rs +++ b/src/meta-srv/src/selector/lease_based.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use api::v1::meta::Peer; +use common_meta::peer::Peer; use crate::error::Result; use crate::lease; diff --git a/src/meta-srv/src/selector/load_based.rs b/src/meta-srv/src/selector/load_based.rs index a5f5beeacd35..e8b3dcdf9e97 100644 --- a/src/meta-srv/src/selector/load_based.rs +++ b/src/meta-srv/src/selector/load_based.rs @@ -14,8 +14,8 @@ use std::collections::HashMap; -use api::v1::meta::Peer; use common_meta::key::TableMetadataManager; +use common_meta::peer::Peer; use common_meta::rpc::router::find_leaders; use common_telemetry::{debug, info}; use parking_lot::RwLock; diff --git a/src/meta-srv/src/selector/weight_compute.rs b/src/meta-srv/src/selector/weight_compute.rs index 10e4b50035c4..c897ebf17412 100644 --- a/src/meta-srv/src/selector/weight_compute.rs +++ b/src/meta-srv/src/selector/weight_compute.rs @@ -14,7 +14,7 @@ use std::collections::HashMap; -use api::v1::meta::Peer; +use common_meta::peer::Peer; use itertools::{Itertools, MinMaxResult}; use crate::keys::{StatKey, StatValue}; @@ -92,7 +92,7 @@ impl WeightCompute for RegionNumsBasedWeightCompute { mod tests { use std::collections::HashMap; - use api::v1::meta::Peer; + use common_meta::peer::Peer; use store_api::region_engine::RegionRole; use store_api::storage::RegionId; diff --git a/src/meta-srv/src/table_meta_alloc.rs b/src/meta-srv/src/table_meta_alloc.rs index 21e5778209f7..636db1b7d6b2 100644 --- a/src/meta-srv/src/table_meta_alloc.rs +++ b/src/meta-srv/src/table_meta_alloc.rs @@ -12,154 +12,72 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; - -use common_catalog::consts::METRIC_ENGINE; +use async_trait::async_trait; use common_error::ext::BoxedError; -use common_meta::ddl::{TableMetadata, TableMetadataAllocator, TableMetadataAllocatorContext}; +use common_meta::ddl::table_meta::PeerAllocator; +use common_meta::ddl::TableMetadataAllocatorContext; use common_meta::error::{ExternalSnafu, Result as MetaResult}; -use common_meta::key::table_route::{ - LogicalTableRouteValue, PhysicalTableRouteValue, TableRouteValue, -}; -use common_meta::rpc::ddl::CreateTableTask; -use common_meta::rpc::router::{Region, RegionRoute}; -use common_meta::sequence::SequenceRef; -use common_meta::wal::{allocate_region_wal_options, WalOptionsAllocatorRef}; -use common_meta::ClusterId; -use common_telemetry::debug; +use common_meta::peer::Peer; use snafu::{ensure, ResultExt}; -use store_api::storage::{RegionId, RegionNumber, TableId, MAX_REGION_SEQ}; +use store_api::storage::MAX_REGION_SEQ; use crate::error::{self, Result, TooManyPartitionsSnafu}; use crate::metasrv::{SelectorContext, SelectorRef}; use crate::selector::SelectorOptions; -pub struct MetaSrvTableMetadataAllocator { +pub struct MetasrvPeerAllocator { ctx: SelectorContext, selector: SelectorRef, - table_id_sequence: SequenceRef, - wal_options_allocator: WalOptionsAllocatorRef, } -impl MetaSrvTableMetadataAllocator { - pub fn new( - ctx: SelectorContext, - selector: SelectorRef, - table_id_sequence: SequenceRef, - wal_options_allocator: WalOptionsAllocatorRef, - ) -> Self { - Self { - ctx, - selector, - table_id_sequence, - wal_options_allocator, - } +impl MetasrvPeerAllocator { + pub fn new(ctx: SelectorContext, selector: SelectorRef) -> Self { + Self { ctx, selector } } - async fn create_table_route( + async fn alloc( &self, - cluster_id: ClusterId, - table_id: TableId, - task: &CreateTableTask, - ) -> Result { - let table_route = if task.create_table.engine == METRIC_ENGINE { - TableRouteValue::Logical(LogicalTableRouteValue {}) - } else { - let regions = task.partitions.len(); - - ensure!(regions <= MAX_REGION_SEQ as usize, TooManyPartitionsSnafu); - - let mut peers = self - .selector - .select( - cluster_id, - &self.ctx, - SelectorOptions { - min_required_items: regions, - allow_duplication: true, - }, - ) - .await?; - - ensure!( - peers.len() >= regions, - error::NoEnoughAvailableDatanodeSnafu { - required: regions, - available: peers.len(), - } - ); - - peers.truncate(regions); - - let region_routes = task - .partitions - .iter() - .enumerate() - .map(|(i, partition)| { - let region = Region { - id: RegionId::new(table_id, i as RegionNumber), - partition: Some(partition.clone().into()), - ..Default::default() - }; - - let peer = peers[i % peers.len()].clone(); + ctx: &TableMetadataAllocatorContext, + regions: usize, + ) -> Result> { + ensure!(regions <= MAX_REGION_SEQ as usize, TooManyPartitionsSnafu); + + let mut peers = self + .selector + .select( + ctx.cluster_id, + &self.ctx, + SelectorOptions { + min_required_items: regions, + allow_duplication: true, + }, + ) + .await?; + + ensure!( + peers.len() >= regions, + error::NoEnoughAvailableDatanodeSnafu { + required: regions, + available: peers.len(), + } + ); - RegionRoute { - region, - leader_peer: Some(peer.into()), - ..Default::default() - } - }) - .collect::>(); - TableRouteValue::Physical(PhysicalTableRouteValue::new(region_routes)) - }; - Ok(table_route) - } + peers.truncate(regions); - fn create_wal_options( - &self, - table_route: &TableRouteValue, - ) -> MetaResult> { - match table_route { - TableRouteValue::Physical(x) => { - let region_numbers = x - .region_routes - .iter() - .map(|route| route.region.id.region_number()) - .collect(); - allocate_region_wal_options(region_numbers, &self.wal_options_allocator) - } - TableRouteValue::Logical(_) => Ok(HashMap::new()), - } + Ok(peers) } } -#[async_trait::async_trait] -impl TableMetadataAllocator for MetaSrvTableMetadataAllocator { - async fn create( +#[async_trait] +impl PeerAllocator for MetasrvPeerAllocator { + async fn alloc( &self, ctx: &TableMetadataAllocatorContext, - task: &CreateTableTask, - ) -> MetaResult { - let table_id = self.table_id_sequence.next().await? as TableId; - - let table_route = self - .create_table_route(ctx.cluster_id, table_id, task) + regions: usize, + ) -> MetaResult> { + self.alloc(ctx, regions) .await .map_err(BoxedError::new) - .context(ExternalSnafu)?; - - let region_wal_options = self.create_wal_options(&table_route)?; - - debug!( - "Allocated region wal options {:?} for table {}", - region_wal_options, table_id - ); - - Ok(TableMetadata { - table_id, - table_route, - region_wal_options, - }) + .context(ExternalSnafu) } } diff --git a/tests-integration/src/standalone.rs b/tests-integration/src/standalone.rs index 14afcb2ca19a..072ff2282099 100644 --- a/tests-integration/src/standalone.rs +++ b/tests-integration/src/standalone.rs @@ -19,6 +19,7 @@ use common_base::Plugins; use common_catalog::consts::MIN_USER_TABLE_ID; use common_config::KvBackendConfig; use common_meta::cache_invalidator::DummyCacheInvalidator; +use common_meta::ddl::table_meta::TableMetadataAllocator; use common_meta::ddl_manager::DdlManager; use common_meta::key::TableMetadataManager; use common_meta::region_keeper::MemoryRegionKeeper; @@ -30,7 +31,6 @@ use datanode::config::DatanodeOptions; use datanode::datanode::DatanodeBuilder; use frontend::frontend::FrontendOptions; use frontend::instance::builder::FrontendBuilder; -use frontend::instance::standalone::StandaloneTableMetadataAllocator; use frontend::instance::{FrontendInstance, Instance, StandaloneDatanodeManager}; use crate::test_util::{self, create_tmp_dir_and_datanode_opts, StorageType, TestGuard}; @@ -123,10 +123,8 @@ impl GreptimeDbStandaloneBuilder { wal_meta.clone(), kv_backend.clone(), )); - let table_meta_allocator = Arc::new(StandaloneTableMetadataAllocator::new( - table_id_sequence, - wal_options_allocator.clone(), - )); + let table_meta_allocator = + TableMetadataAllocator::new(table_id_sequence, wal_options_allocator.clone()); let ddl_task_executor = Arc::new( DdlManager::try_new( diff --git a/tests-integration/tests/region_failover.rs b/tests-integration/tests/region_failover.rs index be1ea3ba92f3..c8cfdc12796f 100644 --- a/tests-integration/tests/region_failover.rs +++ b/tests-integration/tests/region_failover.rs @@ -15,11 +15,11 @@ use std::sync::Arc; use std::time::Duration; -use api::v1::meta::Peer; use catalog::kvbackend::{CachedMetaKvBackend, KvBackendCatalogManager}; use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; use common_meta::key::table_route::TableRouteKey; use common_meta::key::{RegionDistribution, TableMetaKey}; +use common_meta::peer::Peer; use common_meta::{distributed_time_constants, RegionIdent}; use common_procedure::{watcher, ProcedureWithId}; use common_query::Output; From 11ae85b1cdb5f72b093ee5a095cf484dd6254266 Mon Sep 17 00:00:00 2001 From: dennis zhuang Date: Fri, 29 Dec 2023 17:22:31 +0800 Subject: [PATCH 07/21] feat: adds information_schema.schemata (#3051) * feat: improve information_schema.columns * feat: adds information_schema.schemata * fix: instance test * fix: comment --- src/catalog/src/information_schema.rs | 7 + src/catalog/src/information_schema/columns.rs | 45 +++- .../src/information_schema/schemata.rs | 210 ++++++++++++++++++ .../src/information_schema/table_names.rs | 1 + src/common/catalog/src/consts.rs | 2 + src/datatypes/src/schema/column_schema.rs | 11 +- tests-integration/src/tests/instance_test.rs | 76 ++++--- .../common/show/show_databases_tables.result | 1 + .../common/system/information_schema.result | 194 +++++++++------- .../common/system/information_schema.sql | 6 + 10 files changed, 434 insertions(+), 119 deletions(-) create mode 100644 src/catalog/src/information_schema/schemata.rs diff --git a/src/catalog/src/information_schema.rs b/src/catalog/src/information_schema.rs index 5bbfb1c6d88f..232c2279d938 100644 --- a/src/catalog/src/information_schema.rs +++ b/src/catalog/src/information_schema.rs @@ -14,6 +14,7 @@ mod columns; mod memory_table; +mod schemata; mod table_names; mod tables; @@ -41,6 +42,7 @@ pub use table_names::*; use self::columns::InformationSchemaColumns; use crate::error::Result; use crate::information_schema::memory_table::{get_schema_columns, MemoryTable}; +use crate::information_schema::schemata::InformationSchemaSchemata; use crate::information_schema::tables::InformationSchemaTables; use crate::CatalogManager; @@ -126,6 +128,7 @@ impl InformationSchemaProvider { fn build_tables(&mut self) { let mut tables = HashMap::new(); tables.insert(TABLES.to_string(), self.build_table(TABLES).unwrap()); + tables.insert(SCHEMATA.to_string(), self.build_table(SCHEMATA).unwrap()); tables.insert(COLUMNS.to_string(), self.build_table(COLUMNS).unwrap()); // Add memory tables @@ -168,6 +171,10 @@ impl InformationSchemaProvider { } CHECK_CONSTRAINTS => setup_memory_table!(CHECK_CONSTRAINTS), EVENTS => setup_memory_table!(EVENTS), + SCHEMATA => Some(Arc::new(InformationSchemaSchemata::new( + self.catalog_name.clone(), + self.catalog_manager.clone(), + )) as _), _ => None, } } diff --git a/src/catalog/src/information_schema/columns.rs b/src/catalog/src/information_schema/columns.rs index 53f338783ad3..27d4921928e8 100644 --- a/src/catalog/src/information_schema/columns.rs +++ b/src/catalog/src/information_schema/columns.rs @@ -51,6 +51,10 @@ const TABLE_NAME: &str = "table_name"; const COLUMN_NAME: &str = "column_name"; const DATA_TYPE: &str = "data_type"; const SEMANTIC_TYPE: &str = "semantic_type"; +const COLUMN_DEFAULT: &str = "column_default"; +const IS_NULLABLE: &str = "is_nullable"; +const COLUMN_TYPE: &str = "column_type"; +const COLUMN_COMMENT: &str = "column_comment"; impl InformationSchemaColumns { pub(super) fn new(catalog_name: String, catalog_manager: Weak) -> Self { @@ -69,6 +73,10 @@ impl InformationSchemaColumns { ColumnSchema::new(COLUMN_NAME, ConcreteDataType::string_datatype(), false), ColumnSchema::new(DATA_TYPE, ConcreteDataType::string_datatype(), false), ColumnSchema::new(SEMANTIC_TYPE, ConcreteDataType::string_datatype(), false), + ColumnSchema::new(COLUMN_DEFAULT, ConcreteDataType::string_datatype(), true), + ColumnSchema::new(IS_NULLABLE, ConcreteDataType::string_datatype(), false), + ColumnSchema::new(COLUMN_TYPE, ConcreteDataType::string_datatype(), false), + ColumnSchema::new(COLUMN_COMMENT, ConcreteDataType::string_datatype(), true), ])) } @@ -126,6 +134,11 @@ struct InformationSchemaColumnsBuilder { column_names: StringVectorBuilder, data_types: StringVectorBuilder, semantic_types: StringVectorBuilder, + + column_defaults: StringVectorBuilder, + is_nullables: StringVectorBuilder, + column_types: StringVectorBuilder, + column_comments: StringVectorBuilder, } impl InformationSchemaColumnsBuilder { @@ -144,6 +157,10 @@ impl InformationSchemaColumnsBuilder { column_names: StringVectorBuilder::with_capacity(42), data_types: StringVectorBuilder::with_capacity(42), semantic_types: StringVectorBuilder::with_capacity(42), + column_defaults: StringVectorBuilder::with_capacity(42), + is_nullables: StringVectorBuilder::with_capacity(42), + column_types: StringVectorBuilder::with_capacity(42), + column_comments: StringVectorBuilder::with_capacity(42), } } @@ -187,9 +204,8 @@ impl InformationSchemaColumnsBuilder { &catalog_name, &schema_name, &table_name, - &column.name, - &column.data_type.name(), semantic_type, + column, ); } } else { @@ -206,16 +222,31 @@ impl InformationSchemaColumnsBuilder { catalog_name: &str, schema_name: &str, table_name: &str, - column_name: &str, - data_type: &str, semantic_type: &str, + column_schema: &ColumnSchema, ) { + let data_type = &column_schema.data_type.name(); + self.catalog_names.push(Some(catalog_name)); self.schema_names.push(Some(schema_name)); self.table_names.push(Some(table_name)); - self.column_names.push(Some(column_name)); + self.column_names.push(Some(&column_schema.name)); self.data_types.push(Some(data_type)); self.semantic_types.push(Some(semantic_type)); + self.column_defaults.push( + column_schema + .default_constraint() + .map(|s| format!("{}", s)) + .as_deref(), + ); + if column_schema.is_nullable() { + self.is_nullables.push(Some("Yes")); + } else { + self.is_nullables.push(Some("No")); + } + self.column_types.push(Some(data_type)); + self.column_comments + .push(column_schema.column_comment().map(|x| x.as_ref())); } fn finish(&mut self) -> Result { @@ -226,6 +257,10 @@ impl InformationSchemaColumnsBuilder { Arc::new(self.column_names.finish()), Arc::new(self.data_types.finish()), Arc::new(self.semantic_types.finish()), + Arc::new(self.column_defaults.finish()), + Arc::new(self.is_nullables.finish()), + Arc::new(self.column_types.finish()), + Arc::new(self.column_comments.finish()), ]; RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu) diff --git a/src/catalog/src/information_schema/schemata.rs b/src/catalog/src/information_schema/schemata.rs new file mode 100644 index 000000000000..8f523f7a9383 --- /dev/null +++ b/src/catalog/src/information_schema/schemata.rs @@ -0,0 +1,210 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::{Arc, Weak}; + +use arrow_schema::SchemaRef as ArrowSchemaRef; +use common_catalog::consts::INFORMATION_SCHEMA_SCHEMATA_TABLE_ID; +use common_error::ext::BoxedError; +use common_query::physical_plan::TaskContext; +use common_recordbatch::adapter::RecordBatchStreamAdapter; +use common_recordbatch::{RecordBatch, SendableRecordBatchStream}; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter; +use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream; +use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatchStream; +use datatypes::prelude::{ConcreteDataType, ScalarVectorBuilder, VectorRef}; +use datatypes::schema::{ColumnSchema, Schema, SchemaRef}; +use datatypes::vectors::StringVectorBuilder; +use snafu::{OptionExt, ResultExt}; +use store_api::storage::TableId; + +use super::SCHEMATA; +use crate::error::{ + CreateRecordBatchSnafu, InternalSnafu, Result, UpgradeWeakCatalogManagerRefSnafu, +}; +use crate::information_schema::InformationTable; +use crate::CatalogManager; + +/// The `information_schema.schemata` table implementation. +pub(super) struct InformationSchemaSchemata { + schema: SchemaRef, + catalog_name: String, + catalog_manager: Weak, +} + +impl InformationSchemaSchemata { + pub(super) fn new(catalog_name: String, catalog_manager: Weak) -> Self { + Self { + schema: Self::schema(), + catalog_name, + catalog_manager, + } + } + + pub(crate) fn schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + ColumnSchema::new("catalog_name", ConcreteDataType::string_datatype(), false), + ColumnSchema::new("schema_name", ConcreteDataType::string_datatype(), false), + ColumnSchema::new( + "default_character_set_name", + ConcreteDataType::string_datatype(), + false, + ), + ColumnSchema::new( + "default_collation_name", + ConcreteDataType::string_datatype(), + false, + ), + ColumnSchema::new("sql_path", ConcreteDataType::string_datatype(), true), + ])) + } + + fn builder(&self) -> InformationSchemaSchemataBuilder { + InformationSchemaSchemataBuilder::new( + self.schema.clone(), + self.catalog_name.clone(), + self.catalog_manager.clone(), + ) + } +} + +impl InformationTable for InformationSchemaSchemata { + fn table_id(&self) -> TableId { + INFORMATION_SCHEMA_SCHEMATA_TABLE_ID + } + + fn table_name(&self) -> &'static str { + SCHEMATA + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn to_stream(&self) -> Result { + let schema = self.schema.arrow_schema().clone(); + let mut builder = self.builder(); + let stream = Box::pin(DfRecordBatchStreamAdapter::new( + schema, + futures::stream::once(async move { + builder + .make_schemata() + .await + .map(|x| x.into_df_record_batch()) + .map_err(Into::into) + }), + )); + Ok(Box::pin( + RecordBatchStreamAdapter::try_new(stream) + .map_err(BoxedError::new) + .context(InternalSnafu)?, + )) + } +} + +/// Builds the `information_schema.schemata` table row by row +/// +/// Columns are based on +struct InformationSchemaSchemataBuilder { + schema: SchemaRef, + catalog_name: String, + catalog_manager: Weak, + + catalog_names: StringVectorBuilder, + schema_names: StringVectorBuilder, + charset_names: StringVectorBuilder, + collation_names: StringVectorBuilder, + sql_paths: StringVectorBuilder, +} + +impl InformationSchemaSchemataBuilder { + fn new( + schema: SchemaRef, + catalog_name: String, + catalog_manager: Weak, + ) -> Self { + Self { + schema, + catalog_name, + catalog_manager, + catalog_names: StringVectorBuilder::with_capacity(42), + schema_names: StringVectorBuilder::with_capacity(42), + charset_names: StringVectorBuilder::with_capacity(42), + collation_names: StringVectorBuilder::with_capacity(42), + sql_paths: StringVectorBuilder::with_capacity(42), + } + } + + /// Construct the `information_schema.schemata` virtual table + async fn make_schemata(&mut self) -> Result { + let catalog_name = self.catalog_name.clone(); + let catalog_manager = self + .catalog_manager + .upgrade() + .context(UpgradeWeakCatalogManagerRefSnafu)?; + + for schema_name in catalog_manager.schema_names(&catalog_name).await? { + if !catalog_manager + .schema_exists(&catalog_name, &schema_name) + .await? + { + continue; + } + + self.add_schema(&catalog_name, &schema_name); + } + + self.finish() + } + + fn add_schema(&mut self, catalog_name: &str, schema_name: &str) { + self.catalog_names.push(Some(catalog_name)); + self.schema_names.push(Some(schema_name)); + self.charset_names.push(Some("utf8")); + self.collation_names.push(Some("utf8_bin")); + self.sql_paths.push(None); + } + + fn finish(&mut self) -> Result { + let columns: Vec = vec![ + Arc::new(self.catalog_names.finish()), + Arc::new(self.schema_names.finish()), + Arc::new(self.charset_names.finish()), + Arc::new(self.collation_names.finish()), + Arc::new(self.sql_paths.finish()), + ]; + RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu) + } +} + +impl DfPartitionStream for InformationSchemaSchemata { + fn schema(&self) -> &ArrowSchemaRef { + self.schema.arrow_schema() + } + + fn execute(&self, _: Arc) -> DfSendableRecordBatchStream { + let schema = self.schema.arrow_schema().clone(); + let mut builder = self.builder(); + Box::pin(DfRecordBatchStreamAdapter::new( + schema, + futures::stream::once(async move { + builder + .make_schemata() + .await + .map(|x| x.into_df_record_batch()) + .map_err(Into::into) + }), + )) + } +} diff --git a/src/catalog/src/information_schema/table_names.rs b/src/catalog/src/information_schema/table_names.rs index 996a2e035f48..73ef00b81bd3 100644 --- a/src/catalog/src/information_schema/table_names.rs +++ b/src/catalog/src/information_schema/table_names.rs @@ -25,3 +25,4 @@ pub const COLLATIONS: &str = "collations"; pub const COLLATION_CHARACTER_SET_APPLICABILITY: &str = "collation_character_set_applicability"; pub const CHECK_CONSTRAINTS: &str = "check_constraints"; pub const EVENTS: &str = "events"; +pub const SCHEMATA: &str = "schemata"; diff --git a/src/common/catalog/src/consts.rs b/src/common/catalog/src/consts.rs index d9cb1ba54b50..75c176112db2 100644 --- a/src/common/catalog/src/consts.rs +++ b/src/common/catalog/src/consts.rs @@ -54,6 +54,8 @@ pub const INFORMATION_SCHEMA_COLLATION_CHARACTER_SET_APPLICABILITY_TABLE_ID: u32 pub const INFORMATION_SCHEMA_CHECK_CONSTRAINTS_TABLE_ID: u32 = 12; /// id for information_schema.EVENTS pub const INFORMATION_SCHEMA_EVENTS_TABLE_ID: u32 = 13; +/// id for information_schema.SCHEMATA +pub const INFORMATION_SCHEMA_SCHEMATA_TABLE_ID: u32 = 14; /// ----- End of information_schema tables ----- pub const MITO_ENGINE: &str = "mito"; diff --git a/src/datatypes/src/schema/column_schema.rs b/src/datatypes/src/schema/column_schema.rs index 04f715fbc1f1..17c39ee11bd9 100644 --- a/src/datatypes/src/schema/column_schema.rs +++ b/src/datatypes/src/schema/column_schema.rs @@ -109,6 +109,11 @@ impl ColumnSchema { &mut self.metadata } + /// Retrieve the column comment + pub fn column_comment(&self) -> Option<&String> { + self.metadata.get(COMMENT_KEY) + } + pub fn with_time_index(mut self, is_time_index: bool) -> Self { self.is_time_index = is_time_index; if is_time_index { @@ -315,12 +320,16 @@ mod tests { #[test] fn test_column_schema_with_metadata() { - let metadata = Metadata::from([("k1".to_string(), "v1".to_string())]); + let metadata = Metadata::from([ + ("k1".to_string(), "v1".to_string()), + (COMMENT_KEY.to_string(), "test comment".to_string()), + ]); let column_schema = ColumnSchema::new("test", ConcreteDataType::int32_datatype(), true) .with_metadata(metadata) .with_default_constraint(Some(ColumnDefaultConstraint::null_value())) .unwrap(); assert_eq!("v1", column_schema.metadata().get("k1").unwrap()); + assert_eq!("test comment", column_schema.column_comment().unwrap()); assert!(column_schema .metadata() .get(DEFAULT_CONSTRAINT_KEY) diff --git a/tests-integration/src/tests/instance_test.rs b/tests-integration/src/tests/instance_test.rs index 4ec29090e607..686b98a2ee1c 100644 --- a/tests-integration/src/tests/instance_test.rs +++ b/tests-integration/src/tests/instance_test.rs @@ -1775,45 +1775,53 @@ async fn test_information_schema_dot_columns(instance: Arc) { let output = execute_sql(&instance, sql).await; let expected = "\ -+---------------+--------------------+------------+---------------+-----------+---------------+ -| table_catalog | table_schema | table_name | column_name | data_type | semantic_type | -+---------------+--------------------+------------+---------------+-----------+---------------+ -| greptime | information_schema | columns | table_catalog | String | FIELD | -| greptime | information_schema | columns | table_schema | String | FIELD | -| greptime | information_schema | columns | table_name | String | FIELD | -| greptime | information_schema | columns | column_name | String | FIELD | -| greptime | information_schema | columns | data_type | String | FIELD | -| greptime | information_schema | columns | semantic_type | String | FIELD | -| greptime | public | numbers | number | UInt32 | TAG | -| greptime | information_schema | tables | table_catalog | String | FIELD | -| greptime | information_schema | tables | table_schema | String | FIELD | -| greptime | information_schema | tables | table_name | String | FIELD | -| greptime | information_schema | tables | table_type | String | FIELD | -| greptime | information_schema | tables | table_id | UInt32 | FIELD | -| greptime | information_schema | tables | engine | String | FIELD | -+---------------+--------------------+------------+---------------+-----------+---------------+"; ++---------------+--------------------+------------+----------------+-----------+---------------+ +| table_catalog | table_schema | table_name | column_name | data_type | semantic_type | ++---------------+--------------------+------------+----------------+-----------+---------------+ +| greptime | information_schema | columns | table_catalog | String | FIELD | +| greptime | information_schema | columns | table_schema | String | FIELD | +| greptime | information_schema | columns | table_name | String | FIELD | +| greptime | information_schema | columns | column_name | String | FIELD | +| greptime | information_schema | columns | data_type | String | FIELD | +| greptime | information_schema | columns | semantic_type | String | FIELD | +| greptime | information_schema | columns | column_default | String | FIELD | +| greptime | information_schema | columns | is_nullable | String | FIELD | +| greptime | information_schema | columns | column_type | String | FIELD | +| greptime | information_schema | columns | column_comment | String | FIELD | +| greptime | public | numbers | number | UInt32 | TAG | +| greptime | information_schema | tables | table_catalog | String | FIELD | +| greptime | information_schema | tables | table_schema | String | FIELD | +| greptime | information_schema | tables | table_name | String | FIELD | +| greptime | information_schema | tables | table_type | String | FIELD | +| greptime | information_schema | tables | table_id | UInt32 | FIELD | +| greptime | information_schema | tables | engine | String | FIELD | ++---------------+--------------------+------------+----------------+-----------+---------------+"; check_output_stream(output, expected).await; let output = execute_sql_with(&instance, sql, query_ctx).await; let expected = "\ -+-----------------+--------------------+---------------+---------------+----------------------+---------------+ -| table_catalog | table_schema | table_name | column_name | data_type | semantic_type | -+-----------------+--------------------+---------------+---------------+----------------------+---------------+ -| another_catalog | another_schema | another_table | i | TimestampMillisecond | TIMESTAMP | -| another_catalog | information_schema | columns | table_catalog | String | FIELD | -| another_catalog | information_schema | columns | table_schema | String | FIELD | -| another_catalog | information_schema | columns | table_name | String | FIELD | -| another_catalog | information_schema | columns | column_name | String | FIELD | -| another_catalog | information_schema | columns | data_type | String | FIELD | -| another_catalog | information_schema | columns | semantic_type | String | FIELD | -| another_catalog | information_schema | tables | table_catalog | String | FIELD | -| another_catalog | information_schema | tables | table_schema | String | FIELD | -| another_catalog | information_schema | tables | table_name | String | FIELD | -| another_catalog | information_schema | tables | table_type | String | FIELD | -| another_catalog | information_schema | tables | table_id | UInt32 | FIELD | -| another_catalog | information_schema | tables | engine | String | FIELD | -+-----------------+--------------------+---------------+---------------+----------------------+---------------+"; ++-----------------+--------------------+---------------+----------------+----------------------+---------------+ +| table_catalog | table_schema | table_name | column_name | data_type | semantic_type | ++-----------------+--------------------+---------------+----------------+----------------------+---------------+ +| another_catalog | another_schema | another_table | i | TimestampMillisecond | TIMESTAMP | +| another_catalog | information_schema | columns | table_catalog | String | FIELD | +| another_catalog | information_schema | columns | table_schema | String | FIELD | +| another_catalog | information_schema | columns | table_name | String | FIELD | +| another_catalog | information_schema | columns | column_name | String | FIELD | +| another_catalog | information_schema | columns | data_type | String | FIELD | +| another_catalog | information_schema | columns | semantic_type | String | FIELD | +| another_catalog | information_schema | columns | column_default | String | FIELD | +| another_catalog | information_schema | columns | is_nullable | String | FIELD | +| another_catalog | information_schema | columns | column_type | String | FIELD | +| another_catalog | information_schema | columns | column_comment | String | FIELD | +| another_catalog | information_schema | tables | table_catalog | String | FIELD | +| another_catalog | information_schema | tables | table_schema | String | FIELD | +| another_catalog | information_schema | tables | table_name | String | FIELD | +| another_catalog | information_schema | tables | table_type | String | FIELD | +| another_catalog | information_schema | tables | table_id | UInt32 | FIELD | +| another_catalog | information_schema | tables | engine | String | FIELD | ++-----------------+--------------------+---------------+----------------+----------------------+---------------+"; check_output_stream(output, expected).await; } diff --git a/tests/cases/standalone/common/show/show_databases_tables.result b/tests/cases/standalone/common/show/show_databases_tables.result index 0d43286ccc88..a407564c3b4c 100644 --- a/tests/cases/standalone/common/show/show_databases_tables.result +++ b/tests/cases/standalone/common/show/show_databases_tables.result @@ -30,6 +30,7 @@ show tables; | columns | | engines | | events | +| schemata | | tables | +---------------------------------------+ diff --git a/tests/cases/standalone/common/system/information_schema.result b/tests/cases/standalone/common/system/information_schema.result index c3949ca644ba..c7fc6543ee95 100644 --- a/tests/cases/standalone/common/system/information_schema.result +++ b/tests/cases/standalone/common/system/information_schema.result @@ -22,91 +22,101 @@ order by table_schema, table_name; | greptime | information_schema | columns | LOCAL TEMPORARY | 4 | | | greptime | information_schema | engines | LOCAL TEMPORARY | 5 | | | greptime | information_schema | events | LOCAL TEMPORARY | 13 | | +| greptime | information_schema | schemata | LOCAL TEMPORARY | 14 | | | greptime | information_schema | tables | LOCAL TEMPORARY | 3 | | | greptime | public | numbers | LOCAL TEMPORARY | 2 | test_engine | +---------------+--------------------+---------------------------------------+-----------------+----------+-------------+ select * from information_schema.columns order by table_schema, table_name; -+---------------+--------------------+---------------------------------------+----------------------+-----------+---------------+ -| table_catalog | table_schema | table_name | column_name | data_type | semantic_type | -+---------------+--------------------+---------------------------------------+----------------------+-----------+---------------+ -| greptime | information_schema | build_info | git_branch | String | FIELD | -| greptime | information_schema | build_info | git_commit | String | FIELD | -| greptime | information_schema | build_info | git_commit_short | String | FIELD | -| greptime | information_schema | build_info | git_dirty | String | FIELD | -| greptime | information_schema | build_info | pkg_version | String | FIELD | -| greptime | information_schema | character_sets | maxlen | Int64 | FIELD | -| greptime | information_schema | character_sets | character_set_name | String | FIELD | -| greptime | information_schema | character_sets | default_collate_name | String | FIELD | -| greptime | information_schema | character_sets | description | String | FIELD | -| greptime | information_schema | check_constraints | check_clause | String | FIELD | -| greptime | information_schema | check_constraints | constraint_name | String | FIELD | -| greptime | information_schema | check_constraints | constraint_schema | String | FIELD | -| greptime | information_schema | check_constraints | constraint_catalog | String | FIELD | -| greptime | information_schema | collation_character_set_applicability | character_set_name | String | FIELD | -| greptime | information_schema | collation_character_set_applicability | collation_name | String | FIELD | -| greptime | information_schema | collations | collation_name | String | FIELD | -| greptime | information_schema | collations | character_set_name | String | FIELD | -| greptime | information_schema | collations | id | Int64 | FIELD | -| greptime | information_schema | collations | is_default | String | FIELD | -| greptime | information_schema | collations | is_compiled | String | FIELD | -| greptime | information_schema | collations | sortlen | Int64 | FIELD | -| greptime | information_schema | column_privileges | table_catalog | String | FIELD | -| greptime | information_schema | column_privileges | grantee | String | FIELD | -| greptime | information_schema | column_privileges | privilege_type | String | FIELD | -| greptime | information_schema | column_privileges | is_grantable | String | FIELD | -| greptime | information_schema | column_privileges | column_name | String | FIELD | -| greptime | information_schema | column_privileges | table_name | String | FIELD | -| greptime | information_schema | column_privileges | table_schema | String | FIELD | -| greptime | information_schema | column_statistics | schema_name | String | FIELD | -| greptime | information_schema | column_statistics | table_name | String | FIELD | -| greptime | information_schema | column_statistics | column_name | String | FIELD | -| greptime | information_schema | column_statistics | histogram | String | FIELD | -| greptime | information_schema | columns | table_catalog | String | FIELD | -| greptime | information_schema | columns | table_schema | String | FIELD | -| greptime | information_schema | columns | semantic_type | String | FIELD | -| greptime | information_schema | columns | data_type | String | FIELD | -| greptime | information_schema | columns | column_name | String | FIELD | -| greptime | information_schema | columns | table_name | String | FIELD | -| greptime | information_schema | engines | savepoints | String | FIELD | -| greptime | information_schema | engines | xa | String | FIELD | -| greptime | information_schema | engines | transactions | String | FIELD | -| greptime | information_schema | engines | comment | String | FIELD | -| greptime | information_schema | engines | support | String | FIELD | -| greptime | information_schema | engines | engine | String | FIELD | -| greptime | information_schema | events | sql_mode | String | FIELD | -| greptime | information_schema | events | interval_value | Int64 | FIELD | -| greptime | information_schema | events | database_collation | String | FIELD | -| greptime | information_schema | events | collation_connection | String | FIELD | -| greptime | information_schema | events | character_set_client | String | FIELD | -| greptime | information_schema | events | originator | Int64 | FIELD | -| greptime | information_schema | events | event_catalog | String | FIELD | -| greptime | information_schema | events | event_schema | String | FIELD | -| greptime | information_schema | events | event_name | String | FIELD | -| greptime | information_schema | events | definer | String | FIELD | -| greptime | information_schema | events | time_zone | String | FIELD | -| greptime | information_schema | events | event_body | String | FIELD | -| greptime | information_schema | events | event_definition | String | FIELD | -| greptime | information_schema | events | event_type | String | FIELD | -| greptime | information_schema | events | execute_at | DateTime | FIELD | -| greptime | information_schema | events | event_comment | String | FIELD | -| greptime | information_schema | events | interval_field | String | FIELD | -| greptime | information_schema | events | last_executed | DateTime | FIELD | -| greptime | information_schema | events | starts | DateTime | FIELD | -| greptime | information_schema | events | ends | DateTime | FIELD | -| greptime | information_schema | events | status | String | FIELD | -| greptime | information_schema | events | on_completion | String | FIELD | -| greptime | information_schema | events | created | DateTime | FIELD | -| greptime | information_schema | events | last_altered | DateTime | FIELD | -| greptime | information_schema | tables | table_catalog | String | FIELD | -| greptime | information_schema | tables | table_schema | String | FIELD | -| greptime | information_schema | tables | table_name | String | FIELD | -| greptime | information_schema | tables | table_type | String | FIELD | -| greptime | information_schema | tables | table_id | UInt32 | FIELD | -| greptime | information_schema | tables | engine | String | FIELD | -| greptime | public | numbers | number | UInt32 | TAG | -+---------------+--------------------+---------------------------------------+----------------------+-----------+---------------+ ++---------------+--------------------+---------------------------------------+----------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ +| table_catalog | table_schema | table_name | column_name | data_type | semantic_type | column_default | is_nullable | column_type | column_comment | ++---------------+--------------------+---------------------------------------+----------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ +| greptime | information_schema | build_info | pkg_version | String | FIELD | | No | String | | +| greptime | information_schema | build_info | git_branch | String | FIELD | | No | String | | +| greptime | information_schema | build_info | git_commit | String | FIELD | | No | String | | +| greptime | information_schema | build_info | git_commit_short | String | FIELD | | No | String | | +| greptime | information_schema | build_info | git_dirty | String | FIELD | | No | String | | +| greptime | information_schema | character_sets | description | String | FIELD | | No | String | | +| greptime | information_schema | character_sets | maxlen | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | character_sets | default_collate_name | String | FIELD | | No | String | | +| greptime | information_schema | character_sets | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | constraint_schema | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | check_clause | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | constraint_name | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | constraint_catalog | String | FIELD | | No | String | | +| greptime | information_schema | collation_character_set_applicability | collation_name | String | FIELD | | No | String | | +| greptime | information_schema | collation_character_set_applicability | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | collations | collation_name | String | FIELD | | No | String | | +| greptime | information_schema | collations | sortlen | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | collations | is_compiled | String | FIELD | | No | String | | +| greptime | information_schema | collations | is_default | String | FIELD | | No | String | | +| greptime | information_schema | collations | id | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | collations | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | is_grantable | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | grantee | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | table_name | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | column_name | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | privilege_type | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | table_name | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | schema_name | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | column_name | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | histogram | String | FIELD | | No | String | | +| greptime | information_schema | columns | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_comment | String | FIELD | | Yes | String | | +| greptime | information_schema | columns | column_type | String | FIELD | | No | String | | +| greptime | information_schema | columns | is_nullable | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_default | String | FIELD | | Yes | String | | +| greptime | information_schema | columns | semantic_type | String | FIELD | | No | String | | +| greptime | information_schema | columns | data_type | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_name | String | FIELD | | No | String | | +| greptime | information_schema | columns | table_name | String | FIELD | | No | String | | +| greptime | information_schema | columns | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | engines | xa | String | FIELD | | No | String | | +| greptime | information_schema | engines | support | String | FIELD | | No | String | | +| greptime | information_schema | engines | savepoints | String | FIELD | | No | String | | +| greptime | information_schema | engines | engine | String | FIELD | | No | String | | +| greptime | information_schema | engines | transactions | String | FIELD | | No | String | | +| greptime | information_schema | engines | comment | String | FIELD | | No | String | | +| greptime | information_schema | events | event_comment | String | FIELD | | No | String | | +| greptime | information_schema | events | sql_mode | String | FIELD | | No | String | | +| greptime | information_schema | events | database_collation | String | FIELD | | No | String | | +| greptime | information_schema | events | collation_connection | String | FIELD | | No | String | | +| greptime | information_schema | events | character_set_client | String | FIELD | | No | String | | +| greptime | information_schema | events | originator | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | events | event_catalog | String | FIELD | | No | String | | +| greptime | information_schema | events | event_schema | String | FIELD | | No | String | | +| greptime | information_schema | events | event_name | String | FIELD | | No | String | | +| greptime | information_schema | events | definer | String | FIELD | | No | String | | +| greptime | information_schema | events | time_zone | String | FIELD | | No | String | | +| greptime | information_schema | events | event_body | String | FIELD | | No | String | | +| greptime | information_schema | events | event_definition | String | FIELD | | No | String | | +| greptime | information_schema | events | event_type | String | FIELD | | No | String | | +| greptime | information_schema | events | execute_at | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | interval_value | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | events | interval_field | String | FIELD | | No | String | | +| greptime | information_schema | events | last_executed | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | starts | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | ends | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | status | String | FIELD | | No | String | | +| greptime | information_schema | events | on_completion | String | FIELD | | No | String | | +| greptime | information_schema | events | created | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | last_altered | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | schemata | catalog_name | String | FIELD | | No | String | | +| greptime | information_schema | schemata | schema_name | String | FIELD | | No | String | | +| greptime | information_schema | schemata | default_character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | schemata | default_collation_name | String | FIELD | | No | String | | +| greptime | information_schema | schemata | sql_path | String | FIELD | | Yes | String | | +| greptime | information_schema | tables | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_name | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_type | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_id | UInt32 | FIELD | | Yes | UInt32 | | +| greptime | information_schema | tables | engine | String | FIELD | | Yes | String | | +| greptime | public | numbers | number | UInt32 | TAG | | No | UInt32 | | ++---------------+--------------------+---------------------------------------+----------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ create database my_db; @@ -173,6 +183,32 @@ use information_schema; Affected Rows: 0 +-- schemata -- +desc table schemata; + ++----------------------------+--------+-----+------+---------+---------------+ +| Column | Type | Key | Null | Default | Semantic Type | ++----------------------------+--------+-----+------+---------+---------------+ +| catalog_name | String | | NO | | FIELD | +| schema_name | String | | NO | | FIELD | +| default_character_set_name | String | | NO | | FIELD | +| default_collation_name | String | | NO | | FIELD | +| sql_path | String | | YES | | FIELD | ++----------------------------+--------+-----+------+---------+---------------+ + +select * from schemata where catalog_name = 'greptime' and schema_name != 'public' order by catalog_name, schema_name; + ++--------------+-----------------------+----------------------------+------------------------+----------+ +| catalog_name | schema_name | default_character_set_name | default_collation_name | sql_path | ++--------------+-----------------------+----------------------------+------------------------+----------+ +| greptime | greptime_private | utf8 | utf8_bin | | +| greptime | illegal-database | utf8 | utf8_bin | | +| greptime | information_schema | utf8 | utf8_bin | | +| greptime | my_db | utf8 | utf8_bin | | +| greptime | test_public_schema | utf8 | utf8_bin | | +| greptime | upper_case_table_name | utf8 | utf8_bin | | ++--------------+-----------------------+----------------------------+------------------------+----------+ + -- test engines select * from engines; diff --git a/tests/cases/standalone/common/system/information_schema.sql b/tests/cases/standalone/common/system/information_schema.sql index 34db6c144f4f..6550fb544c61 100644 --- a/tests/cases/standalone/common/system/information_schema.sql +++ b/tests/cases/standalone/common/system/information_schema.sql @@ -44,6 +44,12 @@ drop schema my_db; use information_schema; +-- schemata -- + +desc table schemata; + +select * from schemata where catalog_name = 'greptime' and schema_name != 'public' order by catalog_name, schema_name; + -- test engines select * from engines; From 41e51d4ab30843af8b821fde3ea2a42b27580feb Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Fri, 29 Dec 2023 20:17:34 +0800 Subject: [PATCH 08/21] chore: attempt to add doc issue in label task (#3021) * chore: attempt to add doc issue in label task * ci: check pr body for doc issue creation --- .github/workflows/doc-label.yml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/.github/workflows/doc-label.yml b/.github/workflows/doc-label.yml index 930134674abd..d9060c30bf34 100644 --- a/.github/workflows/doc-label.yml +++ b/.github/workflows/doc-label.yml @@ -18,3 +18,14 @@ jobs: enable-versioned-regex: false repo-token: ${{ secrets.GITHUB_TOKEN }} sync-labels: 1 + - name: create an issue in doc repo + uses: dacbd/create-issue-action@main + if: ${{ contains(github.event.pull_request.body, '- [ ] This PR does not require documentation updates.') }} + with: + owner: GreptimeTeam + repo: docs + token: ${{ secrets.DOCS_REPO_TOKEN }} + title: Update docs for ${{ github.event.issue.title || github.event.pull_request.title }} + body: | + A document change request is generated from + ${{ github.event.issue.html_url || github.event.pull_request.html_url }} From 1c94d4c506c3dcf6b221333071faf1c0e40446a8 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Sat, 30 Dec 2023 13:36:14 +0800 Subject: [PATCH 09/21] ci: fix duplicatd doc issue (#3056) --- .github/workflows/doc-label.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/doc-label.yml b/.github/workflows/doc-label.yml index d9060c30bf34..4e547c9b8062 100644 --- a/.github/workflows/doc-label.yml +++ b/.github/workflows/doc-label.yml @@ -20,7 +20,7 @@ jobs: sync-labels: 1 - name: create an issue in doc repo uses: dacbd/create-issue-action@main - if: ${{ contains(github.event.pull_request.body, '- [ ] This PR does not require documentation updates.') }} + if: ${{ github.event.action == 'opened' && contains(github.event.pull_request.body, '- [ ] This PR does not require documentation updates.') }} with: owner: GreptimeTeam repo: docs From 69a53130c223a821bf8ca1b9a028667c5a1c9bf9 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Sat, 30 Dec 2023 15:32:32 +0800 Subject: [PATCH 10/21] feat(inverted_index): Add applier builder to convert Expr to Predicates (Part 1) (#3034) * feat(inverted_index.integration): Add applier builder to convert Expr to Predicates (Part 1) Signed-off-by: Zhenchi * chore: add docs Signed-off-by: Zhenchi * fix: typos Signed-off-by: Zhenchi * fix: address comments Signed-off-by: Zhenchi * Update src/mito2/src/sst/index/applier/builder.rs Co-authored-by: Yingwen * fix: remove unwrap Signed-off-by: Zhenchi * chore: error source Signed-off-by: Zhenchi --------- Signed-off-by: Zhenchi Co-authored-by: Yingwen --- Cargo.lock | 13 +- Cargo.toml | 3 +- src/common/config/src/wal/kafka.rs | 2 +- src/index/src/inverted_index/error.rs | 2 +- .../search/fst_apply/intersection_apply.rs | 112 ++++---- src/mito2/Cargo.toml | 1 + src/mito2/src/error.rs | 20 ++ src/mito2/src/row_converter.rs | 6 +- src/mito2/src/sst.rs | 1 + src/mito2/src/sst/index.rs | 18 ++ src/mito2/src/sst/index/applier.rs | 47 ++++ src/mito2/src/sst/index/applier/builder.rs | 261 ++++++++++++++++++ .../src/sst/index/applier/builder/between.rs | 171 ++++++++++++ src/mito2/src/sst/index/codec.rs | 65 +++++ 14 files changed, 670 insertions(+), 52 deletions(-) create mode 100644 src/mito2/src/sst/index.rs create mode 100644 src/mito2/src/sst/index/applier.rs create mode 100644 src/mito2/src/sst/index/applier/builder.rs create mode 100644 src/mito2/src/sst/index/applier/builder/between.rs create mode 100644 src/mito2/src/sst/index/codec.rs diff --git a/Cargo.lock b/Cargo.lock index d179ea6c8c54..b042227a293b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4029,7 +4029,7 @@ dependencies = [ "prost 0.12.3", "rand", "regex", - "regex-automata 0.1.10", + "regex-automata 0.2.0", "snafu", "tokio", "tokio-util", @@ -4977,6 +4977,7 @@ dependencies = [ "datatypes", "futures", "humantime-serde", + "index", "lazy_static", "log-store", "memcomparable", @@ -7134,8 +7135,18 @@ name = "regex-automata" version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" +dependencies = [ + "regex-syntax 0.6.29", +] + +[[package]] +name = "regex-automata" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e9368763f5a9b804326f3af749e16f9abf378d227bcdee7634b13d8f17793782" dependencies = [ "fst", + "memchr", "regex-syntax 0.6.29", ] diff --git a/Cargo.toml b/Cargo.toml index 0e38d914eccb..a3413aa9d48d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -111,7 +111,7 @@ prost = "0.12" raft-engine = { git = "https://github.com/tikv/raft-engine.git", rev = "22dfb426cd994602b57725ef080287d3e53db479" } rand = "0.8" regex = "1.8" -regex-automata = { version = "0.1", features = ["transducer"] } +regex-automata = { version = "0.2", features = ["transducer"] } reqwest = { version = "0.11", default-features = false, features = [ "json", "rustls-tls-native-roots", @@ -169,6 +169,7 @@ datanode = { path = "src/datanode" } datatypes = { path = "src/datatypes" } file-engine = { path = "src/file-engine" } frontend = { path = "src/frontend" } +index = { path = "src/index" } log-store = { path = "src/log-store" } meta-client = { path = "src/meta-client" } meta-srv = { path = "src/meta-srv" } diff --git a/src/common/config/src/wal/kafka.rs b/src/common/config/src/wal/kafka.rs index e93aa6cb2271..858991264bb6 100644 --- a/src/common/config/src/wal/kafka.rs +++ b/src/common/config/src/wal/kafka.rs @@ -42,7 +42,7 @@ pub struct KafkaConfig { #[serde(skip)] #[serde(default)] pub compression: RsKafkaCompression, - /// The maximum log size a kakfa batch producer could buffer. + /// The maximum log size a kafka batch producer could buffer. pub max_batch_size: ReadableSize, /// The linger duration of a kafka batch producer. #[serde(with = "humantime_serde")] diff --git a/src/index/src/inverted_index/error.rs b/src/index/src/inverted_index/error.rs index b795e33003b7..6e5f39006eb9 100644 --- a/src/index/src/inverted_index/error.rs +++ b/src/index/src/inverted_index/error.rs @@ -113,7 +113,7 @@ pub enum Error { #[snafu(display("Failed to parse regex DFA"))] ParseDFA { #[snafu(source)] - error: regex_automata::Error, + error: Box, location: Location, }, diff --git a/src/index/src/inverted_index/search/fst_apply/intersection_apply.rs b/src/index/src/inverted_index/search/fst_apply/intersection_apply.rs index a0ae0d7b9afb..a608acd0bab5 100644 --- a/src/index/src/inverted_index/search/fst_apply/intersection_apply.rs +++ b/src/index/src/inverted_index/search/fst_apply/intersection_apply.rs @@ -14,7 +14,7 @@ use fst::map::OpBuilder; use fst::{IntoStreamer, Streamer}; -use regex_automata::DenseDFA; +use regex_automata::dfa::dense::DFA; use snafu::{ensure, ResultExt}; use crate::inverted_index::error::{ @@ -24,15 +24,13 @@ use crate::inverted_index::search::fst_apply::FstApplier; use crate::inverted_index::search::predicate::{Predicate, Range}; use crate::inverted_index::FstMap; -type Dfa = DenseDFA, usize>; - /// `IntersectionFstApplier` applies intersection operations on an FstMap using specified ranges and regex patterns. pub struct IntersectionFstApplier { /// A list of `Range` which define inclusive or exclusive ranges for keys to be queried in the FstMap. ranges: Vec, /// A list of `Dfa` compiled from regular expression patterns. - dfas: Vec, + dfas: Vec>>, } impl FstApplier for IntersectionFstApplier { @@ -88,8 +86,8 @@ impl IntersectionFstApplier { match predicate { Predicate::Range(range) => ranges.push(range.range), Predicate::RegexMatch(regex) => { - let dfa = DenseDFA::new(®ex.pattern); - let dfa = dfa.context(ParseDFASnafu)?; + let dfa = DFA::new(®ex.pattern); + let dfa = dfa.map_err(Box::new).context(ParseDFASnafu)?; dfas.push(dfa); } // Rejection of `InList` predicates is enforced here. @@ -210,47 +208,67 @@ mod tests { #[test] fn test_intersection_fst_applier_with_valid_pattern() { - let test_fst = FstMap::from_iter([("aa", 1), ("bb", 2), ("cc", 3)]).unwrap(); - - let applier = create_applier_from_pattern("a.?").unwrap(); - let results = applier.apply(&test_fst); - assert_eq!(results, vec![1]); - - let applier = create_applier_from_pattern("b.?").unwrap(); - let results = applier.apply(&test_fst); - assert_eq!(results, vec![2]); - - let applier = create_applier_from_pattern("c.?").unwrap(); - let results = applier.apply(&test_fst); - assert_eq!(results, vec![3]); - - let applier = create_applier_from_pattern("a.*").unwrap(); - let results = applier.apply(&test_fst); - assert_eq!(results, vec![1]); - - let applier = create_applier_from_pattern("b.*").unwrap(); - let results = applier.apply(&test_fst); - assert_eq!(results, vec![2]); - - let applier = create_applier_from_pattern("c.*").unwrap(); - let results = applier.apply(&test_fst); - assert_eq!(results, vec![3]); - - let applier = create_applier_from_pattern("d.?").unwrap(); - let results = applier.apply(&test_fst); - assert!(results.is_empty()); - - let applier = create_applier_from_pattern("a.?|b.?").unwrap(); - let results = applier.apply(&test_fst); - assert_eq!(results, vec![1, 2]); - - let applier = create_applier_from_pattern("d.?|a.?").unwrap(); - let results = applier.apply(&test_fst); - assert_eq!(results, vec![1]); - - let applier = create_applier_from_pattern(".*").unwrap(); - let results = applier.apply(&test_fst); - assert_eq!(results, vec![1, 2, 3]); + let test_fst = FstMap::from_iter([("123", 1), ("abc", 2)]).unwrap(); + + let cases = vec![ + ("1", vec![1]), + ("2", vec![1]), + ("3", vec![1]), + ("^1", vec![1]), + ("^2", vec![]), + ("^3", vec![]), + ("^1.*", vec![1]), + ("^.*2", vec![1]), + ("^.*3", vec![1]), + ("1$", vec![]), + ("2$", vec![]), + ("3$", vec![1]), + ("1.*$", vec![1]), + ("2.*$", vec![1]), + ("3.*$", vec![1]), + ("^1..$", vec![1]), + ("^.2.$", vec![1]), + ("^..3$", vec![1]), + ("^[0-9]", vec![1]), + ("^[0-9]+$", vec![1]), + ("^[0-9][0-9]$", vec![]), + ("^[0-9][0-9][0-9]$", vec![1]), + ("^123$", vec![1]), + ("a", vec![2]), + ("b", vec![2]), + ("c", vec![2]), + ("^a", vec![2]), + ("^b", vec![]), + ("^c", vec![]), + ("^a.*", vec![2]), + ("^.*b", vec![2]), + ("^.*c", vec![2]), + ("a$", vec![]), + ("b$", vec![]), + ("c$", vec![2]), + ("a.*$", vec![2]), + ("b.*$", vec![2]), + ("c.*$", vec![2]), + ("^.[a-z]", vec![2]), + ("^abc$", vec![2]), + ("^ab$", vec![]), + ("abc$", vec![2]), + ("^a.c$", vec![2]), + ("^..c$", vec![2]), + ("ab", vec![2]), + (".*", vec![1, 2]), + ("", vec![1, 2]), + ("^$", vec![]), + ("1|a", vec![1, 2]), + ("^123$|^abc$", vec![1, 2]), + ("^123$|d", vec![1]), + ]; + + for (pattern, expected) in cases { + let applier = create_applier_from_pattern(pattern).unwrap(); + let results = applier.apply(&test_fst); + assert_eq!(results, expected); + } } #[test] diff --git a/src/mito2/Cargo.toml b/src/mito2/Cargo.toml index 8c3ef50ec2c7..a28e4f0426ea 100644 --- a/src/mito2/Cargo.toml +++ b/src/mito2/Cargo.toml @@ -39,6 +39,7 @@ datafusion.workspace = true datatypes.workspace = true futures.workspace = true humantime-serde.workspace = true +index.workspace = true lazy_static = "1.4" log-store = { workspace = true, optional = true } memcomparable = "0.2" diff --git a/src/mito2/src/error.rs b/src/mito2/src/error.rs index 39457281d76b..68a35123ea39 100644 --- a/src/mito2/src/error.rs +++ b/src/mito2/src/error.rs @@ -423,6 +423,23 @@ pub enum Error { #[snafu(source)] error: parquet::errors::ParquetError, }, + + #[snafu(display("Column not found, column: {column}"))] + ColumnNotFound { column: String, location: Location }, + + #[snafu(display("Failed to build index applier"))] + BuildIndexApplier { + #[snafu(source)] + source: index::inverted_index::error::Error, + location: Location, + }, + + #[snafu(display("Failed to convert value"))] + ConvertValue { + #[snafu(source)] + source: datatypes::error::Error, + location: Location, + }, } pub type Result = std::result::Result; @@ -468,6 +485,7 @@ impl ErrorExt for Error { | InvalidRequest { .. } | FillDefault { .. } | ConvertColumnDataType { .. } + | ColumnNotFound { .. } | InvalidMetadata { .. } => StatusCode::InvalidArguments, RegionMetadataNotFound { .. } | Join { .. } @@ -504,6 +522,8 @@ impl ErrorExt for Error { JsonOptions { .. } => StatusCode::InvalidArguments, EmptyRegionDir { .. } | EmptyManifestDir { .. } => StatusCode::RegionNotFound, ArrowReader { .. } => StatusCode::StorageUnavailable, + BuildIndexApplier { source, .. } => source.status_code(), + ConvertValue { source, .. } => source.status_code(), } } diff --git a/src/mito2/src/row_converter.rs b/src/mito2/src/row_converter.rs index 4cc6fd3274ac..33ef05433521 100644 --- a/src/mito2/src/row_converter.rs +++ b/src/mito2/src/row_converter.rs @@ -84,7 +84,11 @@ impl SortField { } impl SortField { - fn serialize(&self, serializer: &mut Serializer<&mut Vec>, value: &ValueRef) -> Result<()> { + pub(crate) fn serialize( + &self, + serializer: &mut Serializer<&mut Vec>, + value: &ValueRef, + ) -> Result<()> { macro_rules! cast_value_and_serialize { ( $self: ident; diff --git a/src/mito2/src/sst.rs b/src/mito2/src/sst.rs index 32c7b4951a55..55939c2d246a 100644 --- a/src/mito2/src/sst.rs +++ b/src/mito2/src/sst.rs @@ -16,5 +16,6 @@ pub mod file; pub mod file_purger; +mod index; pub mod parquet; pub(crate) mod version; diff --git a/src/mito2/src/sst/index.rs b/src/mito2/src/sst/index.rs new file mode 100644 index 000000000000..baffda27aa6e --- /dev/null +++ b/src/mito2/src/sst/index.rs @@ -0,0 +1,18 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#![allow(dead_code)] + +pub mod applier; +mod codec; diff --git a/src/mito2/src/sst/index/applier.rs b/src/mito2/src/sst/index/applier.rs new file mode 100644 index 000000000000..95ca25ba003d --- /dev/null +++ b/src/mito2/src/sst/index/applier.rs @@ -0,0 +1,47 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +pub mod builder; + +use index::inverted_index::search::index_apply::IndexApplier; +use object_store::ObjectStore; + +/// The [`SstIndexApplier`] is responsible for applying predicates to the provided SST files +/// and returning the relevant row group ids for further scan. +pub struct SstIndexApplier { + /// The root directory of the region. + region_dir: String, + + /// Object store responsible for accessing SST files. + object_store: ObjectStore, + + /// Predefined index applier used to apply predicates to index files + /// and return the relevant row group ids for further scan. + index_applier: Box, +} + +impl SstIndexApplier { + /// Creates a new [`SstIndexApplier`]. + pub fn new( + region_dir: String, + object_store: ObjectStore, + index_applier: Box, + ) -> Self { + Self { + region_dir, + object_store, + index_applier, + } + } +} diff --git a/src/mito2/src/sst/index/applier/builder.rs b/src/mito2/src/sst/index/applier/builder.rs new file mode 100644 index 000000000000..52af22effb18 --- /dev/null +++ b/src/mito2/src/sst/index/applier/builder.rs @@ -0,0 +1,261 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod between; + +// TODO(zhongzc): This PR is too large. The following modules are coming soon. + +// mod comparison; +// mod eq_list; +// mod in_list; +// mod regex_match; + +use std::collections::HashMap; + +use api::v1::SemanticType; +use common_query::logical_plan::Expr; +use common_telemetry::warn; +use datafusion_common::ScalarValue; +use datafusion_expr::Expr as DfExpr; +use datatypes::data_type::ConcreteDataType; +use datatypes::value::Value; +use index::inverted_index::search::index_apply::PredicatesIndexApplier; +use index::inverted_index::search::predicate::Predicate; +use object_store::ObjectStore; +use snafu::{OptionExt, ResultExt}; +use store_api::metadata::RegionMetadata; + +use crate::error::{BuildIndexApplierSnafu, ColumnNotFoundSnafu, ConvertValueSnafu, Result}; +use crate::row_converter::SortField; +use crate::sst::index::applier::SstIndexApplier; +use crate::sst::index::codec::IndexValueCodec; + +type ColumnName = String; + +/// Constructs an [`SstIndexApplier`] which applies predicates to SST files during scan. +pub struct SstIndexApplierBuilder<'a> { + /// Directory of the region, required argument for constructing [`SstIndexApplier`]. + region_dir: String, + + /// Object store, required argument for constructing [`SstIndexApplier`]. + object_store: ObjectStore, + + /// Metadata of the region, used to get metadata like column type. + metadata: &'a RegionMetadata, + + /// Stores predicates during traversal on the Expr tree. + output: HashMap>, +} + +impl<'a> SstIndexApplierBuilder<'a> { + /// Creates a new [`SstIndexApplierBuilder`]. + pub fn new( + region_dir: String, + object_store: ObjectStore, + metadata: &'a RegionMetadata, + ) -> Self { + Self { + region_dir, + object_store, + metadata, + output: HashMap::default(), + } + } + + /// Consumes the builder to construct an [`SstIndexApplier`], optionally returned based on + /// the expressions provided. If no predicates match, returns `None`. + pub fn build(mut self, exprs: &[Expr]) -> Result> { + for expr in exprs { + self.traverse_and_collect(expr.df_expr()); + } + + if self.output.is_empty() { + return Ok(None); + } + + let predicates = self.output.into_iter().collect(); + let applier = PredicatesIndexApplier::try_from(predicates); + Ok(Some(SstIndexApplier::new( + self.region_dir, + self.object_store, + Box::new(applier.context(BuildIndexApplierSnafu)?), + ))) + } + + /// Recursively traverses expressions to collect predicates. + /// Results are stored in `self.output`. + fn traverse_and_collect(&mut self, expr: &DfExpr) { + let res = match expr { + DfExpr::Between(between) => self.collect_between(between), + + // TODO(zhongzc): This PR is too large. The following arms are coming soon. + + // DfExpr::InList(in_list) => self.collect_inlist(in_list), + // DfExpr::BinaryExpr(BinaryExpr { left, op, right }) => match op { + // Operator::And => { + // self.traverse_and_collect(left); + // self.traverse_and_collect(right); + // Ok(()) + // } + // Operator::Or => self.collect_or_eq_list(left, right), + // Operator::Eq => self.collect_eq(left, right), + // Operator::Lt | Operator::LtEq | Operator::Gt | Operator::GtEq => { + // self.collect_comparison_expr(left, op, right) + // } + // Operator::RegexMatch => self.collect_regex_match(left, right), + // _ => Ok(()), + // }, + + // TODO(zhongzc): support more expressions, e.g. IsNull, IsNotNull, ... + _ => Ok(()), + }; + + if let Err(err) = res { + warn!(err; "Failed to collect predicates, ignore it. expr: {expr}"); + } + } + + /// Helper function to add a predicate to the output. + fn add_predicate(&mut self, column_name: &str, predicate: Predicate) { + match self.output.get_mut(column_name) { + Some(predicates) => predicates.push(predicate), + None => { + self.output.insert(column_name.to_string(), vec![predicate]); + } + } + } + + /// Helper function to get the column type of a tag column. + /// Returns `None` if the column is not a tag column. + fn tag_column_type(&self, column_name: &str) -> Result> { + let column = self + .metadata + .column_by_name(column_name) + .context(ColumnNotFoundSnafu { + column: column_name, + })?; + + Ok((column.semantic_type == SemanticType::Tag) + .then(|| column.column_schema.data_type.clone())) + } + + /// Helper function to get a non-null literal. + fn nonnull_lit(expr: &DfExpr) -> Option<&ScalarValue> { + match expr { + DfExpr::Literal(lit) if !lit.is_null() => Some(lit), + _ => None, + } + } + + /// Helper function to get the column name of a column expression. + fn column_name(expr: &DfExpr) -> Option<&str> { + match expr { + DfExpr::Column(column) => Some(&column.name), + _ => None, + } + } + + /// Helper function to encode a literal into bytes. + fn encode_lit(lit: &ScalarValue, data_type: ConcreteDataType) -> Result> { + let value = Value::try_from(lit.clone()).context(ConvertValueSnafu)?; + let mut bytes = vec![]; + let field = SortField::new(data_type); + IndexValueCodec::encode_value(value.as_value_ref(), &field, &mut bytes)?; + Ok(bytes) + } +} + +#[cfg(test)] +mod tests { + use api::v1::SemanticType; + use datafusion_common::Column; + use datatypes::data_type::ConcreteDataType; + use datatypes::schema::ColumnSchema; + use object_store::services::Memory; + use object_store::ObjectStore; + use store_api::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataBuilder}; + use store_api::storage::RegionId; + + use super::*; + + pub(crate) fn test_region_metadata() -> RegionMetadata { + let mut builder = RegionMetadataBuilder::new(RegionId::new(1234, 5678)); + builder + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new("a", ConcreteDataType::string_datatype(), false), + semantic_type: SemanticType::Tag, + column_id: 1, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new("b", ConcreteDataType::string_datatype(), false), + semantic_type: SemanticType::Field, + column_id: 2, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "c", + ConcreteDataType::timestamp_millisecond_datatype(), + false, + ), + semantic_type: SemanticType::Timestamp, + column_id: 3, + }) + .primary_key(vec![1]); + builder.build().unwrap() + } + + pub(crate) fn test_object_store() -> ObjectStore { + ObjectStore::new(Memory::default()).unwrap().finish() + } + + pub(crate) fn tag_column() -> DfExpr { + DfExpr::Column(Column { + relation: None, + name: "a".to_string(), + }) + } + + pub(crate) fn field_column() -> DfExpr { + DfExpr::Column(Column { + relation: None, + name: "b".to_string(), + }) + } + + pub(crate) fn nonexistent_column() -> DfExpr { + DfExpr::Column(Column { + relation: None, + name: "nonexistent".to_string(), + }) + } + + pub(crate) fn string_lit(s: impl Into) -> DfExpr { + DfExpr::Literal(ScalarValue::Utf8(Some(s.into()))) + } + + pub(crate) fn int64_lit(i: impl Into) -> DfExpr { + DfExpr::Literal(ScalarValue::Int64(Some(i.into()))) + } + + pub(crate) fn encoded_string(s: impl Into) -> Vec { + let mut bytes = vec![]; + IndexValueCodec::encode_value( + Value::from(s.into()).as_value_ref(), + &SortField::new(ConcreteDataType::string_datatype()), + &mut bytes, + ) + .unwrap(); + bytes + } +} diff --git a/src/mito2/src/sst/index/applier/builder/between.rs b/src/mito2/src/sst/index/applier/builder/between.rs new file mode 100644 index 000000000000..50ae7073b2db --- /dev/null +++ b/src/mito2/src/sst/index/applier/builder/between.rs @@ -0,0 +1,171 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use datafusion_expr::Between; +use index::inverted_index::search::predicate::{Bound, Predicate, Range, RangePredicate}; + +use crate::error::Result; +use crate::sst::index::applier::builder::SstIndexApplierBuilder; + +impl<'a> SstIndexApplierBuilder<'a> { + /// Collects a `BETWEEN` expression in the form of `column BETWEEN lit AND lit`. + pub(crate) fn collect_between(&mut self, between: &Between) -> Result<()> { + if between.negated { + return Ok(()); + } + + let Some(column_name) = Self::column_name(&between.expr) else { + return Ok(()); + }; + let Some(data_type) = self.tag_column_type(column_name)? else { + return Ok(()); + }; + let Some(low) = Self::nonnull_lit(&between.low) else { + return Ok(()); + }; + let Some(high) = Self::nonnull_lit(&between.high) else { + return Ok(()); + }; + + let predicate = Predicate::Range(RangePredicate { + range: Range { + lower: Some(Bound { + inclusive: true, + value: Self::encode_lit(low, data_type.clone())?, + }), + upper: Some(Bound { + inclusive: true, + value: Self::encode_lit(high, data_type)?, + }), + }, + }); + + self.add_predicate(column_name, predicate); + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::error::Error; + use crate::sst::index::applier::builder::tests::{ + encoded_string, field_column, int64_lit, nonexistent_column, string_lit, tag_column, + test_object_store, test_region_metadata, + }; + + #[test] + fn test_collect_between_basic() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let between = Between { + negated: false, + expr: Box::new(tag_column()), + low: Box::new(string_lit("abc")), + high: Box::new(string_lit("def")), + }; + + builder.collect_between(&between).unwrap(); + + let predicates = builder.output.get("a").unwrap(); + assert_eq!(predicates.len(), 1); + assert_eq!( + predicates[0], + Predicate::Range(RangePredicate { + range: Range { + lower: Some(Bound { + inclusive: true, + value: encoded_string("abc"), + }), + upper: Some(Bound { + inclusive: true, + value: encoded_string("def"), + }), + } + }) + ); + } + + #[test] + fn test_collect_between_negated() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let between = Between { + negated: true, + expr: Box::new(tag_column()), + low: Box::new(string_lit("abc")), + high: Box::new(string_lit("def")), + }; + + builder.collect_between(&between).unwrap(); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_between_field_column() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let between = Between { + negated: false, + expr: Box::new(field_column()), + low: Box::new(string_lit("abc")), + high: Box::new(string_lit("def")), + }; + + builder.collect_between(&between).unwrap(); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_between_type_mismatch() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let between = Between { + negated: false, + expr: Box::new(tag_column()), + low: Box::new(int64_lit(123)), + high: Box::new(int64_lit(456)), + }; + + let res = builder.collect_between(&between); + assert!(matches!(res, Err(Error::FieldTypeMismatch { .. }))); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_between_nonexistent_column() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let between = Between { + negated: false, + expr: Box::new(nonexistent_column()), + low: Box::new(string_lit("abc")), + high: Box::new(string_lit("def")), + }; + + let res = builder.collect_between(&between); + assert!(matches!(res, Err(Error::ColumnNotFound { .. }))); + assert!(builder.output.is_empty()); + } +} diff --git a/src/mito2/src/sst/index/codec.rs b/src/mito2/src/sst/index/codec.rs new file mode 100644 index 000000000000..ada5ac07cbfc --- /dev/null +++ b/src/mito2/src/sst/index/codec.rs @@ -0,0 +1,65 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use datatypes::value::ValueRef; +use memcomparable::Serializer; + +use crate::error::Result; +use crate::row_converter::SortField; + +/// Encodes index values according to their data types for sorting and storage use. +pub struct IndexValueCodec; + +impl IndexValueCodec { + /// Serializes a `ValueRef` using the data type defined in `SortField` and writes + /// the result into a buffer. + /// + /// # Arguments + /// * `value` - The value to be encoded. + /// * `field` - Contains data type to guide serialization. + /// * `buffer` - Destination buffer for the serialized value. + pub fn encode_value(value: ValueRef, field: &SortField, buffer: &mut Vec) -> Result<()> { + buffer.reserve(field.estimated_size()); + let mut serializer = Serializer::new(buffer); + field.serialize(&mut serializer, &value) + } +} + +#[cfg(test)] +mod tests { + use datatypes::data_type::ConcreteDataType; + + use super::*; + use crate::error::Error; + + #[test] + fn test_encode_value_basic() { + let value = ValueRef::from("hello"); + let field = SortField::new(ConcreteDataType::string_datatype()); + + let mut buffer = Vec::new(); + IndexValueCodec::encode_value(value, &field, &mut buffer).unwrap(); + assert!(!buffer.is_empty()); + } + + #[test] + fn test_encode_value_type_mismatch() { + let value = ValueRef::from("hello"); + let field = SortField::new(ConcreteDataType::int64_datatype()); + + let mut buffer = Vec::new(); + let res = IndexValueCodec::encode_value(value, &field, &mut buffer); + assert!(matches!(res, Err(Error::FieldTypeMismatch { .. }))); + } +} From 4460af800f344676f18749ed0a5205ba2e84844b Mon Sep 17 00:00:00 2001 From: AntiTopQuark Date: Sat, 30 Dec 2023 21:02:26 +0800 Subject: [PATCH 11/21] feat(TableRouteValue): add panic notes and type checks (#3031) * refactor(TableRouteValue): add panic notes and type checks * chore: add deprecate develop branch warning Signed-off-by: Ruihang Xia * add error defines and checks * Update README.md * update code format and fix tests * update name of error * delete unused note * fix unsafe .expect() for region_route() * update error name * update unwrap * update code format --------- Signed-off-by: Ruihang Xia Co-authored-by: Ruihang Xia --- src/common/meta/src/ddl/alter_table.rs | 2 +- src/common/meta/src/ddl/create_table.rs | 2 +- src/common/meta/src/ddl/drop_table.rs | 6 +-- src/common/meta/src/ddl_manager.rs | 2 +- src/common/meta/src/error.rs | 6 ++- src/common/meta/src/key.rs | 42 +++++++++------ src/common/meta/src/key/table_route.rs | 54 +++++++++++++------ src/meta-srv/src/error.rs | 10 +++- .../region_failover/deactivate_region.rs | 1 + .../region_failover/update_metadata.rs | 14 +++-- .../src/procedure/region_migration.rs | 2 +- .../src/procedure/region_migration/manager.rs | 3 ++ .../region_migration/migration_start.rs | 5 +- .../procedure/region_migration/test_util.rs | 2 +- .../downgrade_leader_region.rs | 6 +-- .../rollback_downgraded_region.rs | 10 +++- .../upgrade_candidate_region.rs | 16 ++++-- src/meta-srv/src/region/lease_keeper.rs | 2 +- src/meta-srv/src/selector/load_based.rs | 20 ++++--- src/partition/src/error.rs | 8 +++ src/partition/src/manager.rs | 16 ++++-- tests-integration/src/grpc.rs | 14 +++-- tests-integration/src/instance.rs | 14 +++-- 23 files changed, 180 insertions(+), 77 deletions(-) diff --git a/src/common/meta/src/ddl/alter_table.rs b/src/common/meta/src/ddl/alter_table.rs index 092d4dd24263..c3b1f7c31121 100644 --- a/src/common/meta/src/ddl/alter_table.rs +++ b/src/common/meta/src/ddl/alter_table.rs @@ -191,7 +191,7 @@ impl AlterTableProcedure { .await? .context(TableRouteNotFoundSnafu { table_id })? .into_inner(); - let region_routes = table_route.region_routes(); + let region_routes = table_route.region_routes()?; let leaders = find_leaders(region_routes); let mut alter_region_tasks = Vec::with_capacity(leaders.len()); diff --git a/src/common/meta/src/ddl/create_table.rs b/src/common/meta/src/ddl/create_table.rs index c73844fc8337..c6e09006b470 100644 --- a/src/common/meta/src/ddl/create_table.rs +++ b/src/common/meta/src/ddl/create_table.rs @@ -217,7 +217,7 @@ impl CreateTableProcedure { .context(TableRouteNotFoundSnafu { table_id: physical_table_id, })?; - let region_routes = physical_table_route.region_routes(); + let region_routes = physical_table_route.region_routes()?; let request_builder = self.new_region_request_builder(Some(physical_table_id))?; diff --git a/src/common/meta/src/ddl/drop_table.rs b/src/common/meta/src/ddl/drop_table.rs index 94c6cdf0a06a..7fac47e62cb1 100644 --- a/src/common/meta/src/ddl/drop_table.rs +++ b/src/common/meta/src/ddl/drop_table.rs @@ -116,7 +116,7 @@ impl DropTableProcedure { /// Register dropping regions if doesn't exist. fn register_dropping_regions(&mut self) -> Result<()> { - let region_routes = self.data.region_routes(); + let region_routes = self.data.region_routes()?; let dropping_regions = operating_leader_regions(region_routes); @@ -190,7 +190,7 @@ impl DropTableProcedure { pub async fn on_datanode_drop_regions(&self) -> Result { let table_id = self.data.table_id(); - let region_routes = &self.data.region_routes(); + let region_routes = &self.data.region_routes()?; let leaders = find_leaders(region_routes); let mut drop_region_tasks = Vec::with_capacity(leaders.len()); @@ -306,7 +306,7 @@ impl DropTableData { self.task.table_ref() } - fn region_routes(&self) -> &Vec { + fn region_routes(&self) -> Result<&Vec> { self.table_route_value.region_routes() } diff --git a/src/common/meta/src/ddl_manager.rs b/src/common/meta/src/ddl_manager.rs index af669797f4d4..7876d2a8a793 100644 --- a/src/common/meta/src/ddl_manager.rs +++ b/src/common/meta/src/ddl_manager.rs @@ -278,7 +278,7 @@ async fn handle_truncate_table_task( let table_route_value = table_route_value.context(error::TableRouteNotFoundSnafu { table_id })?; - let table_route = table_route_value.into_inner().region_routes().clone(); + let table_route = table_route_value.into_inner().region_routes()?.clone(); let id = ddl_manager .submit_truncate_table_task( diff --git a/src/common/meta/src/error.rs b/src/common/meta/src/error.rs index 323d922b9cda..2a0db2abbb08 100644 --- a/src/common/meta/src/error.rs +++ b/src/common/meta/src/error.rs @@ -351,6 +351,9 @@ pub enum Error { #[snafu(display("The topic pool is empty"))] EmptyTopicPool { location: Location }, + + #[snafu(display("Unexpected table route type: {}", err_msg))] + UnexpectedLogicalRouteTable { location: Location, err_msg: String }, } pub type Result = std::result::Result; @@ -392,7 +395,8 @@ impl ErrorExt for Error { | BuildKafkaPartitionClient { .. } | ProduceRecord { .. } | CreateKafkaWalTopic { .. } - | EmptyTopicPool { .. } => StatusCode::Unexpected, + | EmptyTopicPool { .. } + | UnexpectedLogicalRouteTable { .. } => StatusCode::Unexpected, SendMessage { .. } | GetKvCache { .. } diff --git a/src/common/meta/src/key.rs b/src/common/meta/src/key.rs index bb2b87a973f5..57de421be202 100644 --- a/src/common/meta/src/key.rs +++ b/src/common/meta/src/key.rs @@ -483,7 +483,7 @@ impl TableMetadataManager { .build_delete_txn(table_id, table_info_value)?; // Deletes datanode table key value pairs. - let distribution = region_distribution(table_route_value.region_routes())?; + let distribution = region_distribution(table_route_value.region_routes()?)?; let delete_datanode_txn = self .datanode_table_manager() .build_delete_txn(table_id, distribution)?; @@ -608,7 +608,7 @@ impl TableMetadataManager { ) -> Result<()> { // Updates the datanode table key value pairs. let current_region_distribution = - region_distribution(current_table_route_value.region_routes())?; + region_distribution(current_table_route_value.region_routes()?)?; let new_region_distribution = region_distribution(&new_region_routes)?; let update_datanode_table_txn = self.datanode_table_manager().build_update_txn( @@ -621,7 +621,7 @@ impl TableMetadataManager { )?; // Updates the table_route. - let new_table_route_value = current_table_route_value.update(new_region_routes); + let new_table_route_value = current_table_route_value.update(new_region_routes)?; let (update_table_route_txn, on_update_table_route_failure) = self .table_route_manager() @@ -656,7 +656,7 @@ impl TableMetadataManager { where F: Fn(&RegionRoute) -> Option>, { - let mut new_region_routes = current_table_route_value.region_routes().clone(); + let mut new_region_routes = current_table_route_value.region_routes()?.clone(); let mut updated = 0; for route in &mut new_region_routes { @@ -673,7 +673,7 @@ impl TableMetadataManager { } // Updates the table_route. - let new_table_route_value = current_table_route_value.update(new_region_routes); + let new_table_route_value = current_table_route_value.update(new_region_routes)?; let (update_table_route_txn, on_update_table_route_failure) = self .table_route_manager() @@ -897,7 +897,11 @@ mod tests { table_info ); assert_eq!( - remote_table_route.unwrap().into_inner().region_routes(), + remote_table_route + .unwrap() + .into_inner() + .region_routes() + .unwrap(), region_routes ); } @@ -978,7 +982,7 @@ mod tests { .unwrap() .unwrap() .into_inner(); - assert_eq!(removed_table_route.region_routes(), region_routes); + assert_eq!(removed_table_route.region_routes().unwrap(), region_routes); } #[tokio::test] @@ -1173,11 +1177,11 @@ mod tests { .unwrap(); assert_eq!( - updated_route_value.region_routes()[0].leader_status, + updated_route_value.region_routes().unwrap()[0].leader_status, Some(RegionStatus::Downgraded) ); assert_eq!( - updated_route_value.region_routes()[1].leader_status, + updated_route_value.region_routes().unwrap()[1].leader_status, Some(RegionStatus::Downgraded) ); } @@ -1271,7 +1275,8 @@ mod tests { let current_table_route_value = DeserializedValueWithBytes::from_inner( current_table_route_value .inner - .update(new_region_routes.clone()), + .update(new_region_routes.clone()) + .unwrap(), ); let new_region_routes = vec![new_region_route(2, 4), new_region_route(5, 5)]; // it should be ok. @@ -1295,13 +1300,16 @@ mod tests { // if the current_table_route_value is wrong, it should return an error. // The ABA problem. - let wrong_table_route_value = - DeserializedValueWithBytes::from_inner(current_table_route_value.update(vec![ - new_region_route(1, 1), - new_region_route(2, 2), - new_region_route(3, 3), - new_region_route(4, 4), - ])); + let wrong_table_route_value = DeserializedValueWithBytes::from_inner( + current_table_route_value + .update(vec![ + new_region_route(1, 1), + new_region_route(2, 2), + new_region_route(3, 3), + new_region_route(4, 4), + ]) + .unwrap(), + ); assert!(table_metadata_manager .update_table_route( table_id, diff --git a/src/common/meta/src/key/table_route.rs b/src/common/meta/src/key/table_route.rs index d767d098a79f..4d2ac35001f3 100644 --- a/src/common/meta/src/key/table_route.rs +++ b/src/common/meta/src/key/table_route.rs @@ -16,12 +16,12 @@ use std::collections::HashMap; use std::fmt::Display; use serde::{Deserialize, Serialize}; -use snafu::ResultExt; +use snafu::{ensure, ResultExt}; use store_api::storage::{RegionId, RegionNumber}; use table::metadata::TableId; use super::{DeserializedValueWithBytes, TableMetaValue}; -use crate::error::{Result, SerdeJsonSnafu}; +use crate::error::{Result, SerdeJsonSnafu, UnexpectedLogicalRouteTableSnafu}; use crate::key::{to_removed_key, RegionDistribution, TableMetaKey, TABLE_ROUTE_PREFIX}; use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp, TxnOpResponse}; use crate::kv_backend::KvBackendRef; @@ -62,29 +62,48 @@ impl TableRouteValue { } /// Returns a new version [TableRouteValue] with `region_routes`. - pub fn update(&self, region_routes: Vec) -> Self { + pub fn update(&self, region_routes: Vec) -> Result { + ensure!( + self.is_physical(), + UnexpectedLogicalRouteTableSnafu { + err_msg: "{self:?} is a non-physical TableRouteValue.", + } + ); let version = self.physical_table_route().version; - Self::Physical(PhysicalTableRouteValue { + Ok(Self::Physical(PhysicalTableRouteValue { region_routes, version: version + 1, - }) + })) } /// Returns the version. /// /// For test purpose. #[cfg(any(test, feature = "testing"))] - pub fn version(&self) -> u64 { - self.physical_table_route().version + pub fn version(&self) -> Result { + ensure!( + self.is_physical(), + UnexpectedLogicalRouteTableSnafu { + err_msg: "{self:?} is a non-physical TableRouteValue.", + } + ); + Ok(self.physical_table_route().version) } /// Returns the corresponding [RegionRoute]. - pub fn region_route(&self, region_id: RegionId) -> Option { - self.physical_table_route() + pub fn region_route(&self, region_id: RegionId) -> Result> { + ensure!( + self.is_physical(), + UnexpectedLogicalRouteTableSnafu { + err_msg: "{self:?} is a non-physical TableRouteValue.", + } + ); + Ok(self + .physical_table_route() .region_routes .iter() .find(|route| route.region.id == region_id) - .cloned() + .cloned()) } /// Returns true if it's [TableRouteValue::Physical]. @@ -93,11 +112,14 @@ impl TableRouteValue { } /// Gets the [RegionRoute]s of this [TableRouteValue::Physical]. - /// - /// # Panics - /// The route type is not the [TableRouteValue::Physical]. - pub fn region_routes(&self) -> &Vec { - &self.physical_table_route().region_routes + pub fn region_routes(&self) -> Result<&Vec> { + ensure!( + self.is_physical(), + UnexpectedLogicalRouteTableSnafu { + err_msg: "{self:?} is a non-physical TableRouteValue.", + } + ); + Ok(&self.physical_table_route().region_routes) } fn physical_table_route(&self) -> &PhysicalTableRouteValue { @@ -354,7 +376,7 @@ impl TableRouteManager { ) -> Result> { self.get(table_id) .await? - .map(|table_route| region_distribution(table_route.region_routes())) + .map(|table_route| region_distribution(table_route.region_routes()?)) .transpose() } } diff --git a/src/meta-srv/src/error.rs b/src/meta-srv/src/error.rs index 530fba83aa2e..5272c3abe77a 100644 --- a/src/meta-srv/src/error.rs +++ b/src/meta-srv/src/error.rs @@ -602,6 +602,13 @@ pub enum Error { #[snafu(display("Weight array is not set"))] NotSetWeightArray { location: Location }, + + #[snafu(display("Unexpected table route type: {}", err_msg))] + UnexpectedLogicalRouteTable { + location: Location, + err_msg: String, + source: common_meta::error::Error, + }, } impl Error { @@ -717,7 +724,8 @@ impl ErrorExt for Error { | Error::TableMetadataManager { source, .. } | Error::KvBackend { source, .. } | Error::UpdateTableRoute { source, .. } - | Error::GetFullTableInfo { source, .. } => source.status_code(), + | Error::GetFullTableInfo { source, .. } + | Error::UnexpectedLogicalRouteTable { source, .. } => source.status_code(), Error::InitMetadata { source, .. } | Error::InitDdlManager { source, .. } => { source.status_code() diff --git a/src/meta-srv/src/procedure/region_failover/deactivate_region.rs b/src/meta-srv/src/procedure/region_failover/deactivate_region.rs index c2d06590aec2..650c794126a6 100644 --- a/src/meta-srv/src/procedure/region_failover/deactivate_region.rs +++ b/src/meta-srv/src/procedure/region_failover/deactivate_region.rs @@ -208,6 +208,7 @@ mod tests { let should_downgraded = table_route_value .region_routes() + .unwrap() .iter() .find(|route| route.region.id.region_number() == failed_region.region_number) .unwrap(); diff --git a/src/meta-srv/src/procedure/region_failover/update_metadata.rs b/src/meta-srv/src/procedure/region_failover/update_metadata.rs index 23ade1a2a1fe..c2218c6afede 100644 --- a/src/meta-srv/src/procedure/region_failover/update_metadata.rs +++ b/src/meta-srv/src/procedure/region_failover/update_metadata.rs @@ -85,7 +85,12 @@ impl UpdateRegionMetadata { .context(error::TableMetadataManagerSnafu)? .context(TableRouteNotFoundSnafu { table_id })?; - let mut new_region_routes = table_route_value.region_routes().clone(); + let mut new_region_routes = table_route_value + .region_routes() + .context(error::UnexpectedLogicalRouteTableSnafu { + err_msg: "{self:?} is a non-physical TableRouteValue.", + })? + .clone(); for region_route in new_region_routes.iter_mut() { if region_route.region.id.region_number() == failed_region.region_number { @@ -234,6 +239,7 @@ mod tests { .unwrap() .into_inner() .region_routes() + .unwrap() .clone() } @@ -396,8 +402,8 @@ mod tests { .unwrap() .into_inner(); - let peers = &extract_all_peers(table_route_value.region_routes()); - let actual = table_route_value.region_routes(); + let peers = &extract_all_peers(table_route_value.region_routes().unwrap()); + let actual = table_route_value.region_routes().unwrap(); let expected = &vec![ new_region_route(1, peers, 2), new_region_route(2, peers, 3), @@ -416,7 +422,7 @@ mod tests { .unwrap() .into_inner(); - let map = region_distribution(table_route_value.region_routes()).unwrap(); + let map = region_distribution(table_route_value.region_routes().unwrap()).unwrap(); assert_eq!(map.len(), 2); assert_eq!(map.get(&2), Some(&vec![1, 3])); assert_eq!(map.get(&3), Some(&vec![2, 4])); diff --git a/src/meta-srv/src/procedure/region_migration.rs b/src/meta-srv/src/procedure/region_migration.rs index a1e92277d60b..b187a026723a 100644 --- a/src/meta-srv/src/procedure/region_migration.rs +++ b/src/meta-srv/src/procedure/region_migration.rs @@ -753,7 +753,7 @@ mod tests { .unwrap() .version(); // Should be unchanged. - assert_eq!(table_routes_version, 0); + assert_eq!(table_routes_version.unwrap(), 0); } #[tokio::test] diff --git a/src/meta-srv/src/procedure/region_migration/manager.rs b/src/meta-srv/src/procedure/region_migration/manager.rs index 03794ed85d11..dd034ba3e7d2 100644 --- a/src/meta-srv/src/procedure/region_migration/manager.rs +++ b/src/meta-srv/src/procedure/region_migration/manager.rs @@ -244,6 +244,9 @@ impl RegionMigrationManager { // Safety: checked before. let region_route = table_route .region_route(region_id) + .context(error::UnexpectedLogicalRouteTableSnafu { + err_msg: "{self:?} is a non-physical TableRouteValue.", + })? .context(error::RegionRouteNotFoundSnafu { region_id })?; if self.has_migrated(®ion_route, &task)? { diff --git a/src/meta-srv/src/procedure/region_migration/migration_start.rs b/src/meta-srv/src/procedure/region_migration/migration_start.rs index fa84a1a6dd5e..68b291cb87c1 100644 --- a/src/meta-srv/src/procedure/region_migration/migration_start.rs +++ b/src/meta-srv/src/procedure/region_migration/migration_start.rs @@ -18,7 +18,7 @@ use common_meta::peer::Peer; use common_meta::rpc::router::RegionRoute; use common_procedure::Status; use serde::{Deserialize, Serialize}; -use snafu::OptionExt; +use snafu::{OptionExt, ResultExt}; use store_api::storage::RegionId; use super::migration_end::RegionMigrationEnd; @@ -85,6 +85,9 @@ impl RegionMigrationStart { let region_route = table_route .region_routes() + .context(error::UnexpectedLogicalRouteTableSnafu { + err_msg: "{self:?} is a non-physical TableRouteValue.", + })? .iter() .find(|route| route.region.id == region_id) .cloned() diff --git a/src/meta-srv/src/procedure/region_migration/test_util.rs b/src/meta-srv/src/procedure/region_migration/test_util.rs index 4431791ff70f..4e9bb3939525 100644 --- a/src/meta-srv/src/procedure/region_migration/test_util.rs +++ b/src/meta-srv/src/procedure/region_migration/test_util.rs @@ -419,7 +419,7 @@ impl ProcedureMigrationTestSuite { .unwrap() .unwrap() .into_inner(); - let region_routes = table_route.region_routes(); + let region_routes = table_route.region_routes().unwrap(); let expected_leader_id = self.context.persistent_ctx.to_peer.id; let removed_follower_id = self.context.persistent_ctx.from_peer.id; diff --git a/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs b/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs index cc67aa7ca8e9..818aadd9cda6 100644 --- a/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs +++ b/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs @@ -208,8 +208,8 @@ mod tests { .unwrap(); // It should remain unchanged. - assert_eq!(latest_table_route.version(), 0); - assert!(!latest_table_route.region_routes()[0].is_leader_downgraded()); + assert_eq!(latest_table_route.version().unwrap(), 0); + assert!(!latest_table_route.region_routes().unwrap()[0].is_leader_downgraded()); assert!(ctx.volatile_ctx.table_route.is_none()); } @@ -249,7 +249,7 @@ mod tests { .unwrap() .unwrap(); - assert!(latest_table_route.region_routes()[0].is_leader_downgraded()); + assert!(latest_table_route.region_routes().unwrap()[0].is_leader_downgraded()); assert!(ctx.volatile_ctx.table_route.is_none()); } } diff --git a/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs b/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs index 7281737752a4..844188f2f1f9 100644 --- a/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs +++ b/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs @@ -170,7 +170,10 @@ mod tests { .unwrap() .unwrap() .into_inner(); - assert_eq!(&expected_region_routes, table_route.region_routes()); + assert_eq!( + &expected_region_routes, + table_route.region_routes().unwrap() + ); } #[tokio::test] @@ -231,6 +234,9 @@ mod tests { .unwrap() .unwrap() .into_inner(); - assert_eq!(&expected_region_routes, table_route.region_routes()); + assert_eq!( + &expected_region_routes, + table_route.region_routes().unwrap() + ); } } diff --git a/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs b/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs index 597d9afe9a7b..745b8487a8f3 100644 --- a/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs +++ b/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs @@ -33,7 +33,12 @@ impl UpdateMetadata { let region_id = ctx.region_id(); let table_route_value = ctx.get_table_route_value().await?.clone(); - let mut region_routes = table_route_value.region_routes().clone(); + let mut region_routes = table_route_value + .region_routes() + .context(error::UnexpectedLogicalRouteTableSnafu { + err_msg: "{self:?} is a non-physical TableRouteValue.", + })? + .clone(); let region_route = region_routes .iter_mut() .find(|route| route.region.id == region_id) @@ -81,7 +86,12 @@ impl UpdateMetadata { let region_id = ctx.region_id(); let table_route_value = ctx.get_table_route_value().await?.clone(); - let region_routes = table_route_value.region_routes().clone(); + let region_routes = table_route_value + .region_routes() + .context(error::UnexpectedLogicalRouteTableSnafu { + err_msg: "{self:?} is a non-physical TableRouteValue.", + })? + .clone(); let region_route = region_routes .into_iter() .find(|route| route.region.id == region_id) @@ -465,7 +475,7 @@ mod tests { .unwrap() .unwrap() .into_inner(); - let region_routes = table_route.region_routes(); + let region_routes = table_route.region_routes().unwrap(); assert!(ctx.volatile_ctx.table_route.is_none()); assert!(ctx.volatile_ctx.opening_region_guard.is_none()); diff --git a/src/meta-srv/src/region/lease_keeper.rs b/src/meta-srv/src/region/lease_keeper.rs index cbd2451896b1..9b066065b427 100644 --- a/src/meta-srv/src/region/lease_keeper.rs +++ b/src/meta-srv/src/region/lease_keeper.rs @@ -127,7 +127,7 @@ impl RegionLeaseKeeper { } if let Some(table_route) = table_metadata.get(®ion_id.table_id()) { - if let Some(region_route) = table_route.region_route(region_id) { + if let Ok(Some(region_route)) = table_route.region_route(region_id) { return renew_region_lease_via_region_route(®ion_route, datanode_id, region_id); } } diff --git a/src/meta-srv/src/selector/load_based.rs b/src/meta-srv/src/selector/load_based.rs index e8b3dcdf9e97..9573757a3ffc 100644 --- a/src/meta-srv/src/selector/load_based.rs +++ b/src/meta-srv/src/selector/load_based.rs @@ -142,13 +142,19 @@ async fn get_leader_peer_ids( .await .context(error::TableMetadataManagerSnafu) .map(|route| { - route.map_or_else(Vec::new, |route| { - find_leaders(route.region_routes()) - .into_iter() - .map(|peer| peer.id) - .collect() - }) - }) + route.map_or_else( + || Ok(Vec::new()), + |route| { + let region_routes = route + .region_routes() + .context(error::UnexpectedLogicalRouteTableSnafu { err_msg: "" })?; + Ok(find_leaders(region_routes) + .into_iter() + .map(|peer| peer.id) + .collect()) + }, + ) + })? } #[cfg(test)] diff --git a/src/partition/src/error.rs b/src/partition/src/error.rs index 7765a77c9796..6bfe76fa5b8f 100644 --- a/src/partition/src/error.rs +++ b/src/partition/src/error.rs @@ -119,6 +119,13 @@ pub enum Error { region_id: RegionId, location: Location, }, + + #[snafu(display("Unexpected table route type: {}", err_msg))] + UnexpectedLogicalRouteTable { + location: Location, + err_msg: String, + source: common_meta::error::Error, + }, } impl ErrorExt for Error { @@ -138,6 +145,7 @@ impl ErrorExt for Error { Error::FindDatanode { .. } => StatusCode::InvalidArguments, Error::TableRouteManager { source, .. } => source.status_code(), Error::MissingDefaultValue { .. } => StatusCode::Internal, + Error::UnexpectedLogicalRouteTable { source, .. } => source.status_code(), } } diff --git a/src/partition/src/manager.rs b/src/partition/src/manager.rs index ad15c62cc1dd..2963ac8e2b45 100644 --- a/src/partition/src/manager.rs +++ b/src/partition/src/manager.rs @@ -75,8 +75,13 @@ impl PartitionRuleManager { .context(error::TableRouteManagerSnafu)? .context(error::FindTableRoutesSnafu { table_id })? .into_inner(); - - Ok(RegionRoutes(route.region_routes().clone())) + let region_routes = + route + .region_routes() + .context(error::UnexpectedLogicalRouteTableSnafu { + err_msg: "{self:?} is a non-physical TableRouteValue.", + })?; + Ok(RegionRoutes(region_routes.clone())) } pub async fn find_table_partitions(&self, table_id: TableId) -> Result> { @@ -87,7 +92,12 @@ impl PartitionRuleManager { .context(error::TableRouteManagerSnafu)? .context(error::FindTableRoutesSnafu { table_id })? .into_inner(); - let region_routes = route.region_routes(); + let region_routes = + route + .region_routes() + .context(error::UnexpectedLogicalRouteTableSnafu { + err_msg: "{self:?} is a non-physical TableRouteValue.", + })?; ensure!( !region_routes.is_empty(), diff --git a/tests-integration/src/grpc.rs b/tests-integration/src/grpc.rs index e9731cc336fa..24cd470c3905 100644 --- a/tests-integration/src/grpc.rs +++ b/tests-integration/src/grpc.rs @@ -521,11 +521,15 @@ CREATE TABLE {table_name} ( .unwrap() .into_inner(); - let region_to_dn_map = region_distribution(table_route_value.region_routes()) - .unwrap() - .iter() - .map(|(k, v)| (v[0], *k)) - .collect::>(); + let region_to_dn_map = region_distribution( + table_route_value + .region_routes() + .expect("physical table route"), + ) + .unwrap() + .iter() + .map(|(k, v)| (v[0], *k)) + .collect::>(); assert!(region_to_dn_map.len() <= instance.datanodes().len()); let stmt = QueryLanguageParser::parse_sql(&format!( diff --git a/tests-integration/src/instance.rs b/tests-integration/src/instance.rs index 05253dc0a236..5b7ed080d9d9 100644 --- a/tests-integration/src/instance.rs +++ b/tests-integration/src/instance.rs @@ -216,11 +216,15 @@ mod tests { .unwrap() .into_inner(); - let region_to_dn_map = region_distribution(table_route_value.region_routes()) - .unwrap() - .iter() - .map(|(k, v)| (v[0], *k)) - .collect::>(); + let region_to_dn_map = region_distribution( + table_route_value + .region_routes() + .expect("region routes should be physical"), + ) + .unwrap() + .iter() + .map(|(k, v)| (v[0], *k)) + .collect::>(); assert!(region_to_dn_map.len() <= instance.datanodes().len()); let stmt = QueryLanguageParser::parse_sql("SELECT ts, host FROM demo ORDER BY ts").unwrap(); From 9db168875c4f67be5df6103d0c79f5fd159b3f0e Mon Sep 17 00:00:00 2001 From: niebayes Date: Sat, 30 Dec 2023 23:28:10 +0800 Subject: [PATCH 12/21] fix(remote_wal): some known issues (#3052) * fix: some known issues * fix: CR * fix: CR * chore: replace Mutex with RwLock --- src/log-store/src/kafka/client_manager.rs | 28 ++++++++++------ src/log-store/src/kafka/log_store.rs | 39 +++++++++++------------ 2 files changed, 38 insertions(+), 29 deletions(-) diff --git a/src/log-store/src/kafka/client_manager.rs b/src/log-store/src/kafka/client_manager.rs index e272840201bb..cd2f705c4db9 100644 --- a/src/log-store/src/kafka/client_manager.rs +++ b/src/log-store/src/kafka/client_manager.rs @@ -12,17 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::sync::Arc; use common_config::wal::{KafkaConfig, KafkaWalTopic as Topic}; -use dashmap::mapref::entry::Entry as DashMapEntry; -use dashmap::DashMap; use rskafka::client::partition::{PartitionClient, UnknownTopicHandling}; use rskafka::client::producer::aggregator::RecordAggregator; use rskafka::client::producer::{BatchProducer, BatchProducerBuilder}; use rskafka::client::{Client as RsKafkaClient, ClientBuilder}; use rskafka::BackoffConfig; use snafu::ResultExt; +use tokio::sync::RwLock; use crate::error::{BuildClientSnafu, BuildPartitionClientSnafu, Result}; @@ -67,7 +67,7 @@ pub(crate) struct ClientManager { client_factory: RsKafkaClient, /// A pool maintaining a collection of clients. /// Key: a topic. Value: the associated client of the topic. - client_pool: DashMap, + client_pool: RwLock>, } impl ClientManager { @@ -91,18 +91,28 @@ impl ClientManager { Ok(Self { config: config.clone(), client_factory: client, - client_pool: DashMap::new(), + client_pool: RwLock::new(HashMap::new()), }) } /// Gets the client associated with the topic. If the client does not exist, a new one will /// be created and returned. pub(crate) async fn get_or_insert(&self, topic: &Topic) -> Result { - match self.client_pool.entry(topic.to_string()) { - DashMapEntry::Occupied(entry) => Ok(entry.get().clone()), - DashMapEntry::Vacant(entry) => { - let topic_client = self.try_create_client(topic).await?; - Ok(entry.insert(topic_client).clone()) + let client_pool = self.client_pool.read().await; + if let Some(client) = client_pool.get(topic) { + return Ok(client.clone()); + } + // Manullay releases the read lock. + drop(client_pool); + + // Acquires the write lock. + let mut client_pool = self.client_pool.write().await; + match client_pool.get(topic) { + Some(client) => Ok(client.clone()), + None => { + let client = self.try_create_client(topic).await?; + client_pool.insert(topic.clone(), client.clone()); + Ok(client) } } } diff --git a/src/log-store/src/kafka/log_store.rs b/src/log-store/src/kafka/log_store.rs index 36c86987041b..20bcd4e7cf50 100644 --- a/src/log-store/src/kafka/log_store.rs +++ b/src/log-store/src/kafka/log_store.rs @@ -100,25 +100,24 @@ impl LogStore for KafkaLogStore { .push(entry); } - // Builds a record from entries belong to a region and produces them to kafka server. - let region_ids = producers.keys().cloned().collect::>(); - - let tasks = producers - .into_values() - .map(|producer| producer.produce(&self.client_manager)) - .collect::>(); - // Each produce operation returns a kafka offset of the produced record. - // The offsets are then converted to entry ids. - let entry_ids = futures::future::try_join_all(tasks) - .await? - .into_iter() - .map(TryInto::try_into) - .collect::>>()?; - debug!("The entries are appended at offsets {:?}", entry_ids); - - Ok(AppendBatchResponse { - last_entry_ids: region_ids.into_iter().zip(entry_ids).collect(), - }) + // Produces entries for each region and gets the offset those entries written to. + // The returned offset is then converted into an entry id. + let last_entry_ids = futures::future::try_join_all(producers.into_iter().map( + |(region_id, producer)| async move { + let entry_id = producer + .produce(&self.client_manager) + .await + .map(TryInto::try_into)??; + Ok((region_id, entry_id)) + }, + )) + .await? + .into_iter() + .collect::>(); + + debug!("Append batch result: {:?}", last_entry_ids); + + Ok(AppendBatchResponse { last_entry_ids }) } /// Creates a new `EntryStream` to asynchronously generates `Entry` with entry ids @@ -186,7 +185,7 @@ impl LogStore for KafkaLogStore { record_offset, ns_clone, high_watermark ); - // Ignores the noop record. + // Ignores noop records. if record.record.value.is_none() { continue; } From 6070e880776e0216796a0155420ed7dafb5d18a0 Mon Sep 17 00:00:00 2001 From: dimbtp Date: Sun, 31 Dec 2023 10:08:16 +0800 Subject: [PATCH 13/21] feat: add information_schema.files (#3054) * feat: add information_schema.files * fix: update information_schema.result * fix: change `EXTRA` field type to string --- src/catalog/src/information_schema.rs | 2 + .../information_schema/memory_table/tables.rs | 44 +++++++++ .../src/information_schema/table_names.rs | 1 + src/common/catalog/src/consts.rs | 4 +- .../common/show/show_databases_tables.result | 1 + .../common/system/information_schema.result | 95 +++++++++++++------ 6 files changed, 118 insertions(+), 29 deletions(-) diff --git a/src/catalog/src/information_schema.rs b/src/catalog/src/information_schema.rs index 232c2279d938..92427552425f 100644 --- a/src/catalog/src/information_schema.rs +++ b/src/catalog/src/information_schema.rs @@ -58,6 +58,7 @@ lazy_static! { COLLATION_CHARACTER_SET_APPLICABILITY, CHECK_CONSTRAINTS, EVENTS, + FILES, ]; } @@ -171,6 +172,7 @@ impl InformationSchemaProvider { } CHECK_CONSTRAINTS => setup_memory_table!(CHECK_CONSTRAINTS), EVENTS => setup_memory_table!(EVENTS), + FILES => setup_memory_table!(FILES), SCHEMATA => Some(Arc::new(InformationSchemaSchemata::new( self.catalog_name.clone(), self.catalog_manager.clone(), diff --git a/src/catalog/src/information_schema/memory_table/tables.rs b/src/catalog/src/information_schema/memory_table/tables.rs index abb719ca1b4b..30be1fbaa748 100644 --- a/src/catalog/src/information_schema/memory_table/tables.rs +++ b/src/catalog/src/information_schema/memory_table/tables.rs @@ -183,6 +183,50 @@ pub fn get_schema_columns(table_name: &str) -> (SchemaRef, Vec) { vec![], ), + FILES => ( + vec![ + bigint_column("FILE_ID"), + string_column("FILE_NAME"), + string_column("FILE_TYPE"), + string_column("TABLESPACE_NAME"), + string_column("TABLE_CATALOG"), + string_column("TABLE_SCHEMA"), + string_column("TABLE_NAME"), + string_column("LOGFILE_GROUP_NAME"), + bigint_column("LOGFILE_GROUP_NUMBER"), + string_column("ENGINE"), + string_column("FULLTEXT_KEYS"), + bigint_column("DELETED_ROWS"), + bigint_column("UPDATE_COUNT"), + bigint_column("FREE_EXTENTS"), + bigint_column("TOTAL_EXTENTS"), + bigint_column("EXTENT_SIZE"), + bigint_column("INITIAL_SIZE"), + bigint_column("MAXIMUM_SIZE"), + bigint_column("AUTOEXTEND_SIZE"), + datetime_column("CREATION_TIME"), + datetime_column("LAST_UPDATE_TIME"), + datetime_column("LAST_ACCESS_TIME"), + datetime_column("RECOVER_TIME"), + bigint_column("TRANSACTION_COUNTER"), + string_column("VERSION"), + string_column("ROW_FORMAT"), + bigint_column("TABLE_ROWS"), + bigint_column("AVG_ROW_LENGTH"), + bigint_column("DATA_LENGTH"), + bigint_column("MAX_DATA_LENGTH"), + bigint_column("INDEX_LENGTH"), + bigint_column("DATA_FREE"), + datetime_column("CREATE_TIME"), + datetime_column("UPDATE_TIME"), + datetime_column("CHECK_TIME"), + string_column("CHECKSUM"), + string_column("STATUS"), + string_column("EXTRA"), + ], + vec![], + ), + _ => unreachable!("Unknown table in information_schema: {}", table_name), }; diff --git a/src/catalog/src/information_schema/table_names.rs b/src/catalog/src/information_schema/table_names.rs index 73ef00b81bd3..bfdc56d89217 100644 --- a/src/catalog/src/information_schema/table_names.rs +++ b/src/catalog/src/information_schema/table_names.rs @@ -25,4 +25,5 @@ pub const COLLATIONS: &str = "collations"; pub const COLLATION_CHARACTER_SET_APPLICABILITY: &str = "collation_character_set_applicability"; pub const CHECK_CONSTRAINTS: &str = "check_constraints"; pub const EVENTS: &str = "events"; +pub const FILES: &str = "files"; pub const SCHEMATA: &str = "schemata"; diff --git a/src/common/catalog/src/consts.rs b/src/common/catalog/src/consts.rs index 75c176112db2..9e8b9e4a0768 100644 --- a/src/common/catalog/src/consts.rs +++ b/src/common/catalog/src/consts.rs @@ -54,8 +54,10 @@ pub const INFORMATION_SCHEMA_COLLATION_CHARACTER_SET_APPLICABILITY_TABLE_ID: u32 pub const INFORMATION_SCHEMA_CHECK_CONSTRAINTS_TABLE_ID: u32 = 12; /// id for information_schema.EVENTS pub const INFORMATION_SCHEMA_EVENTS_TABLE_ID: u32 = 13; +/// id for information_schema.FILES +pub const INFORMATION_SCHEMA_FILES_TABLE_ID: u32 = 14; /// id for information_schema.SCHEMATA -pub const INFORMATION_SCHEMA_SCHEMATA_TABLE_ID: u32 = 14; +pub const INFORMATION_SCHEMA_SCHEMATA_TABLE_ID: u32 = 15; /// ----- End of information_schema tables ----- pub const MITO_ENGINE: &str = "mito"; diff --git a/tests/cases/standalone/common/show/show_databases_tables.result b/tests/cases/standalone/common/show/show_databases_tables.result index a407564c3b4c..8a1e606ad110 100644 --- a/tests/cases/standalone/common/show/show_databases_tables.result +++ b/tests/cases/standalone/common/show/show_databases_tables.result @@ -30,6 +30,7 @@ show tables; | columns | | engines | | events | +| files | | schemata | | tables | +---------------------------------------+ diff --git a/tests/cases/standalone/common/system/information_schema.result b/tests/cases/standalone/common/system/information_schema.result index c7fc6543ee95..a97660ee5839 100644 --- a/tests/cases/standalone/common/system/information_schema.result +++ b/tests/cases/standalone/common/system/information_schema.result @@ -22,7 +22,8 @@ order by table_schema, table_name; | greptime | information_schema | columns | LOCAL TEMPORARY | 4 | | | greptime | information_schema | engines | LOCAL TEMPORARY | 5 | | | greptime | information_schema | events | LOCAL TEMPORARY | 13 | | -| greptime | information_schema | schemata | LOCAL TEMPORARY | 14 | | +| greptime | information_schema | files | LOCAL TEMPORARY | 14 | | +| greptime | information_schema | schemata | LOCAL TEMPORARY | 15 | | | greptime | information_schema | tables | LOCAL TEMPORARY | 3 | | | greptime | public | numbers | LOCAL TEMPORARY | 2 | test_engine | +---------------+--------------------+---------------------------------------+-----------------+----------+-------------+ @@ -32,56 +33,56 @@ select * from information_schema.columns order by table_schema, table_name; +---------------+--------------------+---------------------------------------+----------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ | table_catalog | table_schema | table_name | column_name | data_type | semantic_type | column_default | is_nullable | column_type | column_comment | +---------------+--------------------+---------------------------------------+----------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ -| greptime | information_schema | build_info | pkg_version | String | FIELD | | No | String | | | greptime | information_schema | build_info | git_branch | String | FIELD | | No | String | | | greptime | information_schema | build_info | git_commit | String | FIELD | | No | String | | | greptime | information_schema | build_info | git_commit_short | String | FIELD | | No | String | | | greptime | information_schema | build_info | git_dirty | String | FIELD | | No | String | | -| greptime | information_schema | character_sets | description | String | FIELD | | No | String | | -| greptime | information_schema | character_sets | maxlen | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | build_info | pkg_version | String | FIELD | | No | String | | | greptime | information_schema | character_sets | default_collate_name | String | FIELD | | No | String | | | greptime | information_schema | character_sets | character_set_name | String | FIELD | | No | String | | -| greptime | information_schema | check_constraints | constraint_schema | String | FIELD | | No | String | | +| greptime | information_schema | character_sets | maxlen | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | character_sets | description | String | FIELD | | No | String | | | greptime | information_schema | check_constraints | check_clause | String | FIELD | | No | String | | | greptime | information_schema | check_constraints | constraint_name | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | constraint_schema | String | FIELD | | No | String | | | greptime | information_schema | check_constraints | constraint_catalog | String | FIELD | | No | String | | -| greptime | information_schema | collation_character_set_applicability | collation_name | String | FIELD | | No | String | | | greptime | information_schema | collation_character_set_applicability | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | collation_character_set_applicability | collation_name | String | FIELD | | No | String | | | greptime | information_schema | collations | collation_name | String | FIELD | | No | String | | | greptime | information_schema | collations | sortlen | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | collations | is_compiled | String | FIELD | | No | String | | -| greptime | information_schema | collations | is_default | String | FIELD | | No | String | | -| greptime | information_schema | collations | id | Int64 | FIELD | | No | Int64 | | | greptime | information_schema | collations | character_set_name | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | is_grantable | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | grantee | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | collations | id | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | collations | is_default | String | FIELD | | No | String | | +| greptime | information_schema | collations | is_compiled | String | FIELD | | No | String | | | greptime | information_schema | column_privileges | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | grantee | String | FIELD | | No | String | | | greptime | information_schema | column_privileges | table_name | String | FIELD | | No | String | | | greptime | information_schema | column_privileges | column_name | String | FIELD | | No | String | | | greptime | information_schema | column_privileges | privilege_type | String | FIELD | | No | String | | -| greptime | information_schema | column_statistics | table_name | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | is_grantable | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | histogram | String | FIELD | | No | String | | | greptime | information_schema | column_statistics | schema_name | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | table_name | String | FIELD | | No | String | | | greptime | information_schema | column_statistics | column_name | String | FIELD | | No | String | | -| greptime | information_schema | column_statistics | histogram | String | FIELD | | No | String | | -| greptime | information_schema | columns | table_catalog | String | FIELD | | No | String | | -| greptime | information_schema | columns | column_comment | String | FIELD | | Yes | String | | | greptime | information_schema | columns | column_type | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_comment | String | FIELD | | Yes | String | | +| greptime | information_schema | columns | table_schema | String | FIELD | | No | String | | | greptime | information_schema | columns | is_nullable | String | FIELD | | No | String | | | greptime | information_schema | columns | column_default | String | FIELD | | Yes | String | | | greptime | information_schema | columns | semantic_type | String | FIELD | | No | String | | +| greptime | information_schema | columns | table_catalog | String | FIELD | | No | String | | | greptime | information_schema | columns | data_type | String | FIELD | | No | String | | | greptime | information_schema | columns | column_name | String | FIELD | | No | String | | | greptime | information_schema | columns | table_name | String | FIELD | | No | String | | -| greptime | information_schema | columns | table_schema | String | FIELD | | No | String | | -| greptime | information_schema | engines | xa | String | FIELD | | No | String | | -| greptime | information_schema | engines | support | String | FIELD | | No | String | | | greptime | information_schema | engines | savepoints | String | FIELD | | No | String | | -| greptime | information_schema | engines | engine | String | FIELD | | No | String | | +| greptime | information_schema | engines | xa | String | FIELD | | No | String | | | greptime | information_schema | engines | transactions | String | FIELD | | No | String | | | greptime | information_schema | engines | comment | String | FIELD | | No | String | | -| greptime | information_schema | events | event_comment | String | FIELD | | No | String | | -| greptime | information_schema | events | sql_mode | String | FIELD | | No | String | | +| greptime | information_schema | engines | support | String | FIELD | | No | String | | +| greptime | information_schema | engines | engine | String | FIELD | | No | String | | +| greptime | information_schema | events | event_definition | String | FIELD | | No | String | | +| greptime | information_schema | events | last_executed | DateTime | FIELD | | No | DateTime | | | greptime | information_schema | events | database_collation | String | FIELD | | No | String | | | greptime | information_schema | events | collation_connection | String | FIELD | | No | String | | | greptime | information_schema | events | character_set_client | String | FIELD | | No | String | | @@ -92,29 +93,67 @@ select * from information_schema.columns order by table_schema, table_name; | greptime | information_schema | events | definer | String | FIELD | | No | String | | | greptime | information_schema | events | time_zone | String | FIELD | | No | String | | | greptime | information_schema | events | event_body | String | FIELD | | No | String | | -| greptime | information_schema | events | event_definition | String | FIELD | | No | String | | +| greptime | information_schema | events | event_comment | String | FIELD | | No | String | | | greptime | information_schema | events | event_type | String | FIELD | | No | String | | | greptime | information_schema | events | execute_at | DateTime | FIELD | | No | DateTime | | | greptime | information_schema | events | interval_value | Int64 | FIELD | | No | Int64 | | | greptime | information_schema | events | interval_field | String | FIELD | | No | String | | -| greptime | information_schema | events | last_executed | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | sql_mode | String | FIELD | | No | String | | | greptime | information_schema | events | starts | DateTime | FIELD | | No | DateTime | | | greptime | information_schema | events | ends | DateTime | FIELD | | No | DateTime | | | greptime | information_schema | events | status | String | FIELD | | No | String | | | greptime | information_schema | events | on_completion | String | FIELD | | No | String | | | greptime | information_schema | events | created | DateTime | FIELD | | No | DateTime | | | greptime | information_schema | events | last_altered | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | logfile_group_name | String | FIELD | | No | String | | +| greptime | information_schema | files | data_free | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | extra | String | FIELD | | No | String | | +| greptime | information_schema | files | status | String | FIELD | | No | String | | +| greptime | information_schema | files | checksum | String | FIELD | | No | String | | +| greptime | information_schema | files | check_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | file_id | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | file_name | String | FIELD | | No | String | | +| greptime | information_schema | files | file_type | String | FIELD | | No | String | | +| greptime | information_schema | files | tablespace_name | String | FIELD | | No | String | | +| greptime | information_schema | files | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | files | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | files | table_name | String | FIELD | | No | String | | +| greptime | information_schema | files | update_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | logfile_group_number | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | engine | String | FIELD | | No | String | | +| greptime | information_schema | files | fulltext_keys | String | FIELD | | No | String | | +| greptime | information_schema | files | deleted_rows | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | update_count | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | free_extents | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | total_extents | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | extent_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | initial_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | maximum_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | autoextend_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | creation_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | last_update_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | last_access_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | recover_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | transaction_counter | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | version | String | FIELD | | No | String | | +| greptime | information_schema | files | row_format | String | FIELD | | No | String | | +| greptime | information_schema | files | table_rows | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | avg_row_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | data_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | max_data_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | index_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | create_time | DateTime | FIELD | | No | DateTime | | | greptime | information_schema | schemata | catalog_name | String | FIELD | | No | String | | | greptime | information_schema | schemata | schema_name | String | FIELD | | No | String | | | greptime | information_schema | schemata | default_character_set_name | String | FIELD | | No | String | | | greptime | information_schema | schemata | default_collation_name | String | FIELD | | No | String | | | greptime | information_schema | schemata | sql_path | String | FIELD | | Yes | String | | +| greptime | information_schema | tables | engine | String | FIELD | | Yes | String | | | greptime | information_schema | tables | table_catalog | String | FIELD | | No | String | | -| greptime | information_schema | tables | table_schema | String | FIELD | | No | String | | -| greptime | information_schema | tables | table_name | String | FIELD | | No | String | | -| greptime | information_schema | tables | table_type | String | FIELD | | No | String | | | greptime | information_schema | tables | table_id | UInt32 | FIELD | | Yes | UInt32 | | -| greptime | information_schema | tables | engine | String | FIELD | | Yes | String | | +| greptime | information_schema | tables | table_type | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_name | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_schema | String | FIELD | | No | String | | | greptime | public | numbers | number | UInt32 | TAG | | No | UInt32 | | +---------------+--------------------+---------------------------------------+----------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ From f735f739e5de7a028a7b860b4e507eb774c9523a Mon Sep 17 00:00:00 2001 From: dimbtp Date: Sun, 31 Dec 2023 20:29:06 +0800 Subject: [PATCH 14/21] feat: add information_schema.key_column_usage (#3057) * feat: add information_schema.key_column_usage * fix: follow #3057 review comments * fix: add sql test for `key_column_usage` table * fix: fix spell typo * fix: resolve conflict in sql test result --- src/catalog/src/information_schema.rs | 10 + .../information_schema/key_column_usage.rs | 338 ++++++++++++++++++ .../src/information_schema/table_names.rs | 1 + src/common/catalog/src/consts.rs | 2 + .../common/show/show_databases_tables.result | 1 + .../common/system/information_schema.result | 293 ++++++++------- .../common/system/information_schema.sql | 4 + 7 files changed, 523 insertions(+), 126 deletions(-) create mode 100644 src/catalog/src/information_schema/key_column_usage.rs diff --git a/src/catalog/src/information_schema.rs b/src/catalog/src/information_schema.rs index 92427552425f..f806e307b757 100644 --- a/src/catalog/src/information_schema.rs +++ b/src/catalog/src/information_schema.rs @@ -13,6 +13,7 @@ // limitations under the License. mod columns; +mod key_column_usage; mod memory_table; mod schemata; mod table_names; @@ -41,6 +42,7 @@ pub use table_names::*; use self::columns::InformationSchemaColumns; use crate::error::Result; +use crate::information_schema::key_column_usage::InformationSchemaKeyColumnUsage; use crate::information_schema::memory_table::{get_schema_columns, MemoryTable}; use crate::information_schema::schemata::InformationSchemaSchemata; use crate::information_schema::tables::InformationSchemaTables; @@ -131,6 +133,10 @@ impl InformationSchemaProvider { tables.insert(TABLES.to_string(), self.build_table(TABLES).unwrap()); tables.insert(SCHEMATA.to_string(), self.build_table(SCHEMATA).unwrap()); tables.insert(COLUMNS.to_string(), self.build_table(COLUMNS).unwrap()); + tables.insert( + KEY_COLUMN_USAGE.to_string(), + self.build_table(KEY_COLUMN_USAGE).unwrap(), + ); // Add memory tables for name in MEMORY_TABLES.iter() { @@ -173,6 +179,10 @@ impl InformationSchemaProvider { CHECK_CONSTRAINTS => setup_memory_table!(CHECK_CONSTRAINTS), EVENTS => setup_memory_table!(EVENTS), FILES => setup_memory_table!(FILES), + KEY_COLUMN_USAGE => Some(Arc::new(InformationSchemaKeyColumnUsage::new( + self.catalog_name.clone(), + self.catalog_manager.clone(), + )) as _), SCHEMATA => Some(Arc::new(InformationSchemaSchemata::new( self.catalog_name.clone(), self.catalog_manager.clone(), diff --git a/src/catalog/src/information_schema/key_column_usage.rs b/src/catalog/src/information_schema/key_column_usage.rs new file mode 100644 index 000000000000..7c8a4995acfd --- /dev/null +++ b/src/catalog/src/information_schema/key_column_usage.rs @@ -0,0 +1,338 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::{Arc, Weak}; + +use arrow_schema::SchemaRef as ArrowSchemaRef; +use common_catalog::consts::INFORMATION_SCHEMA_KEY_COLUMN_USAGE_TABLE_ID; +use common_error::ext::BoxedError; +use common_query::physical_plan::TaskContext; +use common_recordbatch::adapter::RecordBatchStreamAdapter; +use common_recordbatch::{RecordBatch, SendableRecordBatchStream}; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter; +use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream; +use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatchStream; +use datatypes::prelude::{ConcreteDataType, ScalarVectorBuilder, VectorRef}; +use datatypes::schema::{ColumnSchema, Schema, SchemaRef}; +use datatypes::vectors::{StringVectorBuilder, UInt32VectorBuilder}; +use snafu::{OptionExt, ResultExt}; +use store_api::storage::TableId; + +use super::KEY_COLUMN_USAGE; +use crate::error::{ + CreateRecordBatchSnafu, InternalSnafu, Result, UpgradeWeakCatalogManagerRefSnafu, +}; +use crate::information_schema::InformationTable; +use crate::CatalogManager; + +/// The virtual table implementation for `information_schema.KEY_COLUMN_USAGE`. +pub(super) struct InformationSchemaKeyColumnUsage { + schema: SchemaRef, + catalog_name: String, + catalog_manager: Weak, +} + +impl InformationSchemaKeyColumnUsage { + pub(super) fn new(catalog_name: String, catalog_manager: Weak) -> Self { + Self { + schema: Self::schema(), + catalog_name, + catalog_manager, + } + } + + pub(crate) fn schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + ColumnSchema::new( + "constraint_catalog", + ConcreteDataType::string_datatype(), + false, + ), + ColumnSchema::new( + "constraint_schema", + ConcreteDataType::string_datatype(), + false, + ), + ColumnSchema::new( + "constraint_name", + ConcreteDataType::string_datatype(), + false, + ), + ColumnSchema::new("table_catalog", ConcreteDataType::string_datatype(), false), + ColumnSchema::new("table_schema", ConcreteDataType::string_datatype(), false), + ColumnSchema::new("table_name", ConcreteDataType::string_datatype(), false), + ColumnSchema::new("column_name", ConcreteDataType::string_datatype(), false), + ColumnSchema::new( + "ordinal_position", + ConcreteDataType::uint32_datatype(), + false, + ), + ColumnSchema::new( + "position_in_unique_constraint", + ConcreteDataType::uint32_datatype(), + true, + ), + ColumnSchema::new( + "referenced_table_schema", + ConcreteDataType::string_datatype(), + true, + ), + ColumnSchema::new( + "referenced_table_name", + ConcreteDataType::string_datatype(), + true, + ), + ColumnSchema::new( + "referenced_column_name", + ConcreteDataType::string_datatype(), + true, + ), + ])) + } + + fn builder(&self) -> InformationSchemaKeyColumnUsageBuilder { + InformationSchemaKeyColumnUsageBuilder::new( + self.schema.clone(), + self.catalog_name.clone(), + self.catalog_manager.clone(), + ) + } +} + +impl InformationTable for InformationSchemaKeyColumnUsage { + fn table_id(&self) -> TableId { + INFORMATION_SCHEMA_KEY_COLUMN_USAGE_TABLE_ID + } + + fn table_name(&self) -> &'static str { + KEY_COLUMN_USAGE + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn to_stream(&self) -> Result { + let schema = self.schema.arrow_schema().clone(); + let mut builder = self.builder(); + let stream = Box::pin(DfRecordBatchStreamAdapter::new( + schema, + futures::stream::once(async move { + builder + .make_key_column_usage() + .await + .map(|x| x.into_df_record_batch()) + .map_err(Into::into) + }), + )); + Ok(Box::pin( + RecordBatchStreamAdapter::try_new(stream) + .map_err(BoxedError::new) + .context(InternalSnafu)?, + )) + } +} + +/// Builds the `information_schema.KEY_COLUMN_USAGE` table row by row +/// +/// Columns are based on +struct InformationSchemaKeyColumnUsageBuilder { + schema: SchemaRef, + catalog_name: String, + catalog_manager: Weak, + + constraint_catalog: StringVectorBuilder, + constraint_schema: StringVectorBuilder, + constraint_name: StringVectorBuilder, + table_catalog: StringVectorBuilder, + table_schema: StringVectorBuilder, + table_name: StringVectorBuilder, + column_name: StringVectorBuilder, + ordinal_position: UInt32VectorBuilder, + position_in_unique_constraint: UInt32VectorBuilder, + referenced_table_schema: StringVectorBuilder, + referenced_table_name: StringVectorBuilder, + referenced_column_name: StringVectorBuilder, +} + +impl InformationSchemaKeyColumnUsageBuilder { + fn new( + schema: SchemaRef, + catalog_name: String, + catalog_manager: Weak, + ) -> Self { + Self { + schema, + catalog_name, + catalog_manager, + constraint_catalog: StringVectorBuilder::with_capacity(42), + constraint_schema: StringVectorBuilder::with_capacity(42), + constraint_name: StringVectorBuilder::with_capacity(42), + table_catalog: StringVectorBuilder::with_capacity(42), + table_schema: StringVectorBuilder::with_capacity(42), + table_name: StringVectorBuilder::with_capacity(42), + column_name: StringVectorBuilder::with_capacity(42), + ordinal_position: UInt32VectorBuilder::with_capacity(42), + position_in_unique_constraint: UInt32VectorBuilder::with_capacity(42), + referenced_table_schema: StringVectorBuilder::with_capacity(42), + referenced_table_name: StringVectorBuilder::with_capacity(42), + referenced_column_name: StringVectorBuilder::with_capacity(42), + } + } + + /// Construct the `information_schema.KEY_COLUMN_USAGE` virtual table + async fn make_key_column_usage(&mut self) -> Result { + let catalog_name = self.catalog_name.clone(); + let catalog_manager = self + .catalog_manager + .upgrade() + .context(UpgradeWeakCatalogManagerRefSnafu)?; + + let mut time_index_constraints = vec![]; + let mut primary_constraints = vec![]; + + for schema_name in catalog_manager.schema_names(&catalog_name).await? { + if !catalog_manager + .schema_exists(&catalog_name, &schema_name) + .await? + { + continue; + } + + for table_name in catalog_manager + .table_names(&catalog_name, &schema_name) + .await? + { + if let Some(table) = catalog_manager + .table(&catalog_name, &schema_name, &table_name) + .await? + { + let keys = &table.table_info().meta.primary_key_indices; + let schema = table.schema(); + + for (idx, column) in schema.column_schemas().iter().enumerate() { + if column.is_time_index() { + time_index_constraints.push(( + schema_name.clone(), + table_name.clone(), + column.name.clone(), + )); + } + if keys.contains(&idx) { + primary_constraints.push(( + schema_name.clone(), + table_name.clone(), + column.name.clone(), + )); + } + // TODO(dimbtp): foreign key constraint not supported yet + } + } else { + unreachable!(); + } + } + } + + for (i, (schema_name, table_name, column_name)) in + time_index_constraints.into_iter().enumerate() + { + self.add_key_column_usage( + &schema_name, + "TIME INDEX", + &schema_name, + &table_name, + &column_name, + i as u32 + 1, + ); + } + for (i, (schema_name, table_name, column_name)) in + primary_constraints.into_iter().enumerate() + { + self.add_key_column_usage( + &schema_name, + "PRIMARY", + &schema_name, + &table_name, + &column_name, + i as u32 + 1, + ); + } + + self.finish() + } + + // TODO(dimbtp): Foreign key constraint has not `None` value for last 4 + // fields, but it is not supported yet. + fn add_key_column_usage( + &mut self, + constraint_schema: &str, + constraint_name: &str, + table_schema: &str, + table_name: &str, + column_name: &str, + ordinal_position: u32, + ) { + self.constraint_catalog.push(Some("def")); + self.constraint_schema.push(Some(constraint_schema)); + self.constraint_name.push(Some(constraint_name)); + self.table_catalog.push(Some("def")); + self.table_schema.push(Some(table_schema)); + self.table_name.push(Some(table_name)); + self.column_name.push(Some(column_name)); + self.ordinal_position.push(Some(ordinal_position)); + self.position_in_unique_constraint.push(None); + self.referenced_table_schema.push(None); + self.referenced_table_name.push(None); + self.referenced_column_name.push(None); + } + + fn finish(&mut self) -> Result { + let columns: Vec = vec![ + Arc::new(self.constraint_catalog.finish()), + Arc::new(self.constraint_schema.finish()), + Arc::new(self.constraint_name.finish()), + Arc::new(self.table_catalog.finish()), + Arc::new(self.table_schema.finish()), + Arc::new(self.table_name.finish()), + Arc::new(self.column_name.finish()), + Arc::new(self.ordinal_position.finish()), + Arc::new(self.position_in_unique_constraint.finish()), + Arc::new(self.referenced_table_schema.finish()), + Arc::new(self.referenced_table_name.finish()), + Arc::new(self.referenced_column_name.finish()), + ]; + RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu) + } +} + +impl DfPartitionStream for InformationSchemaKeyColumnUsage { + fn schema(&self) -> &ArrowSchemaRef { + self.schema.arrow_schema() + } + + fn execute(&self, _: Arc) -> DfSendableRecordBatchStream { + let schema = self.schema.arrow_schema().clone(); + let mut builder = self.builder(); + Box::pin(DfRecordBatchStreamAdapter::new( + schema, + futures::stream::once(async move { + builder + .make_key_column_usage() + .await + .map(|x| x.into_df_record_batch()) + .map_err(Into::into) + }), + )) + } +} diff --git a/src/catalog/src/information_schema/table_names.rs b/src/catalog/src/information_schema/table_names.rs index bfdc56d89217..e1b23b4f6938 100644 --- a/src/catalog/src/information_schema/table_names.rs +++ b/src/catalog/src/information_schema/table_names.rs @@ -25,5 +25,6 @@ pub const COLLATIONS: &str = "collations"; pub const COLLATION_CHARACTER_SET_APPLICABILITY: &str = "collation_character_set_applicability"; pub const CHECK_CONSTRAINTS: &str = "check_constraints"; pub const EVENTS: &str = "events"; +pub const KEY_COLUMN_USAGE: &str = "key_column_usage"; pub const FILES: &str = "files"; pub const SCHEMATA: &str = "schemata"; diff --git a/src/common/catalog/src/consts.rs b/src/common/catalog/src/consts.rs index 9e8b9e4a0768..6af76120a5f8 100644 --- a/src/common/catalog/src/consts.rs +++ b/src/common/catalog/src/consts.rs @@ -58,6 +58,8 @@ pub const INFORMATION_SCHEMA_EVENTS_TABLE_ID: u32 = 13; pub const INFORMATION_SCHEMA_FILES_TABLE_ID: u32 = 14; /// id for information_schema.SCHEMATA pub const INFORMATION_SCHEMA_SCHEMATA_TABLE_ID: u32 = 15; +/// id for information_schema.KEY_COLUMN_USAGE +pub const INFORMATION_SCHEMA_KEY_COLUMN_USAGE_TABLE_ID: u32 = 16; /// ----- End of information_schema tables ----- pub const MITO_ENGINE: &str = "mito"; diff --git a/tests/cases/standalone/common/show/show_databases_tables.result b/tests/cases/standalone/common/show/show_databases_tables.result index 8a1e606ad110..76a69d56aa8b 100644 --- a/tests/cases/standalone/common/show/show_databases_tables.result +++ b/tests/cases/standalone/common/show/show_databases_tables.result @@ -31,6 +31,7 @@ show tables; | engines | | events | | files | +| key_column_usage | | schemata | | tables | +---------------------------------------+ diff --git a/tests/cases/standalone/common/system/information_schema.result b/tests/cases/standalone/common/system/information_schema.result index a97660ee5839..a119b028a344 100644 --- a/tests/cases/standalone/common/system/information_schema.result +++ b/tests/cases/standalone/common/system/information_schema.result @@ -23,6 +23,7 @@ order by table_schema, table_name; | greptime | information_schema | engines | LOCAL TEMPORARY | 5 | | | greptime | information_schema | events | LOCAL TEMPORARY | 13 | | | greptime | information_schema | files | LOCAL TEMPORARY | 14 | | +| greptime | information_schema | key_column_usage | LOCAL TEMPORARY | 16 | | | greptime | information_schema | schemata | LOCAL TEMPORARY | 15 | | | greptime | information_schema | tables | LOCAL TEMPORARY | 3 | | | greptime | public | numbers | LOCAL TEMPORARY | 2 | test_engine | @@ -30,132 +31,144 @@ order by table_schema, table_name; select * from information_schema.columns order by table_schema, table_name; -+---------------+--------------------+---------------------------------------+----------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ -| table_catalog | table_schema | table_name | column_name | data_type | semantic_type | column_default | is_nullable | column_type | column_comment | -+---------------+--------------------+---------------------------------------+----------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ -| greptime | information_schema | build_info | git_branch | String | FIELD | | No | String | | -| greptime | information_schema | build_info | git_commit | String | FIELD | | No | String | | -| greptime | information_schema | build_info | git_commit_short | String | FIELD | | No | String | | -| greptime | information_schema | build_info | git_dirty | String | FIELD | | No | String | | -| greptime | information_schema | build_info | pkg_version | String | FIELD | | No | String | | -| greptime | information_schema | character_sets | default_collate_name | String | FIELD | | No | String | | -| greptime | information_schema | character_sets | character_set_name | String | FIELD | | No | String | | -| greptime | information_schema | character_sets | maxlen | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | character_sets | description | String | FIELD | | No | String | | -| greptime | information_schema | check_constraints | check_clause | String | FIELD | | No | String | | -| greptime | information_schema | check_constraints | constraint_name | String | FIELD | | No | String | | -| greptime | information_schema | check_constraints | constraint_schema | String | FIELD | | No | String | | -| greptime | information_schema | check_constraints | constraint_catalog | String | FIELD | | No | String | | -| greptime | information_schema | collation_character_set_applicability | character_set_name | String | FIELD | | No | String | | -| greptime | information_schema | collation_character_set_applicability | collation_name | String | FIELD | | No | String | | -| greptime | information_schema | collations | collation_name | String | FIELD | | No | String | | -| greptime | information_schema | collations | sortlen | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | collations | character_set_name | String | FIELD | | No | String | | -| greptime | information_schema | collations | id | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | collations | is_default | String | FIELD | | No | String | | -| greptime | information_schema | collations | is_compiled | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | table_schema | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | table_catalog | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | grantee | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | table_name | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | column_name | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | privilege_type | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | is_grantable | String | FIELD | | No | String | | -| greptime | information_schema | column_statistics | histogram | String | FIELD | | No | String | | -| greptime | information_schema | column_statistics | schema_name | String | FIELD | | No | String | | -| greptime | information_schema | column_statistics | table_name | String | FIELD | | No | String | | -| greptime | information_schema | column_statistics | column_name | String | FIELD | | No | String | | -| greptime | information_schema | columns | column_type | String | FIELD | | No | String | | -| greptime | information_schema | columns | column_comment | String | FIELD | | Yes | String | | -| greptime | information_schema | columns | table_schema | String | FIELD | | No | String | | -| greptime | information_schema | columns | is_nullable | String | FIELD | | No | String | | -| greptime | information_schema | columns | column_default | String | FIELD | | Yes | String | | -| greptime | information_schema | columns | semantic_type | String | FIELD | | No | String | | -| greptime | information_schema | columns | table_catalog | String | FIELD | | No | String | | -| greptime | information_schema | columns | data_type | String | FIELD | | No | String | | -| greptime | information_schema | columns | column_name | String | FIELD | | No | String | | -| greptime | information_schema | columns | table_name | String | FIELD | | No | String | | -| greptime | information_schema | engines | savepoints | String | FIELD | | No | String | | -| greptime | information_schema | engines | xa | String | FIELD | | No | String | | -| greptime | information_schema | engines | transactions | String | FIELD | | No | String | | -| greptime | information_schema | engines | comment | String | FIELD | | No | String | | -| greptime | information_schema | engines | support | String | FIELD | | No | String | | -| greptime | information_schema | engines | engine | String | FIELD | | No | String | | -| greptime | information_schema | events | event_definition | String | FIELD | | No | String | | -| greptime | information_schema | events | last_executed | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | events | database_collation | String | FIELD | | No | String | | -| greptime | information_schema | events | collation_connection | String | FIELD | | No | String | | -| greptime | information_schema | events | character_set_client | String | FIELD | | No | String | | -| greptime | information_schema | events | originator | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | events | event_catalog | String | FIELD | | No | String | | -| greptime | information_schema | events | event_schema | String | FIELD | | No | String | | -| greptime | information_schema | events | event_name | String | FIELD | | No | String | | -| greptime | information_schema | events | definer | String | FIELD | | No | String | | -| greptime | information_schema | events | time_zone | String | FIELD | | No | String | | -| greptime | information_schema | events | event_body | String | FIELD | | No | String | | -| greptime | information_schema | events | event_comment | String | FIELD | | No | String | | -| greptime | information_schema | events | event_type | String | FIELD | | No | String | | -| greptime | information_schema | events | execute_at | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | events | interval_value | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | events | interval_field | String | FIELD | | No | String | | -| greptime | information_schema | events | sql_mode | String | FIELD | | No | String | | -| greptime | information_schema | events | starts | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | events | ends | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | events | status | String | FIELD | | No | String | | -| greptime | information_schema | events | on_completion | String | FIELD | | No | String | | -| greptime | information_schema | events | created | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | events | last_altered | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | logfile_group_name | String | FIELD | | No | String | | -| greptime | information_schema | files | data_free | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | extra | String | FIELD | | No | String | | -| greptime | information_schema | files | status | String | FIELD | | No | String | | -| greptime | information_schema | files | checksum | String | FIELD | | No | String | | -| greptime | information_schema | files | check_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | file_id | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | file_name | String | FIELD | | No | String | | -| greptime | information_schema | files | file_type | String | FIELD | | No | String | | -| greptime | information_schema | files | tablespace_name | String | FIELD | | No | String | | -| greptime | information_schema | files | table_catalog | String | FIELD | | No | String | | -| greptime | information_schema | files | table_schema | String | FIELD | | No | String | | -| greptime | information_schema | files | table_name | String | FIELD | | No | String | | -| greptime | information_schema | files | update_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | logfile_group_number | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | engine | String | FIELD | | No | String | | -| greptime | information_schema | files | fulltext_keys | String | FIELD | | No | String | | -| greptime | information_schema | files | deleted_rows | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | update_count | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | free_extents | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | total_extents | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | extent_size | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | initial_size | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | maximum_size | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | autoextend_size | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | creation_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | last_update_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | last_access_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | recover_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | transaction_counter | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | version | String | FIELD | | No | String | | -| greptime | information_schema | files | row_format | String | FIELD | | No | String | | -| greptime | information_schema | files | table_rows | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | avg_row_length | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | data_length | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | max_data_length | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | index_length | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | create_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | schemata | catalog_name | String | FIELD | | No | String | | -| greptime | information_schema | schemata | schema_name | String | FIELD | | No | String | | -| greptime | information_schema | schemata | default_character_set_name | String | FIELD | | No | String | | -| greptime | information_schema | schemata | default_collation_name | String | FIELD | | No | String | | -| greptime | information_schema | schemata | sql_path | String | FIELD | | Yes | String | | -| greptime | information_schema | tables | engine | String | FIELD | | Yes | String | | -| greptime | information_schema | tables | table_catalog | String | FIELD | | No | String | | -| greptime | information_schema | tables | table_id | UInt32 | FIELD | | Yes | UInt32 | | -| greptime | information_schema | tables | table_type | String | FIELD | | No | String | | -| greptime | information_schema | tables | table_name | String | FIELD | | No | String | | -| greptime | information_schema | tables | table_schema | String | FIELD | | No | String | | -| greptime | public | numbers | number | UInt32 | TAG | | No | UInt32 | | -+---------------+--------------------+---------------------------------------+----------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ ++---------------+--------------------+---------------------------------------+-------------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ +| table_catalog | table_schema | table_name | column_name | data_type | semantic_type | column_default | is_nullable | column_type | column_comment | ++---------------+--------------------+---------------------------------------+-------------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ +| greptime | information_schema | build_info | git_branch | String | FIELD | | No | String | | +| greptime | information_schema | build_info | git_commit | String | FIELD | | No | String | | +| greptime | information_schema | build_info | git_commit_short | String | FIELD | | No | String | | +| greptime | information_schema | build_info | git_dirty | String | FIELD | | No | String | | +| greptime | information_schema | build_info | pkg_version | String | FIELD | | No | String | | +| greptime | information_schema | character_sets | description | String | FIELD | | No | String | | +| greptime | information_schema | character_sets | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | character_sets | default_collate_name | String | FIELD | | No | String | | +| greptime | information_schema | character_sets | maxlen | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | check_constraints | check_clause | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | constraint_name | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | constraint_schema | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | constraint_catalog | String | FIELD | | No | String | | +| greptime | information_schema | collation_character_set_applicability | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | collation_character_set_applicability | collation_name | String | FIELD | | No | String | | +| greptime | information_schema | collations | collation_name | String | FIELD | | No | String | | +| greptime | information_schema | collations | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | collations | id | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | collations | is_default | String | FIELD | | No | String | | +| greptime | information_schema | collations | is_compiled | String | FIELD | | No | String | | +| greptime | information_schema | collations | sortlen | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | column_privileges | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | grantee | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | table_name | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | column_name | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | privilege_type | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | is_grantable | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | histogram | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | schema_name | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | table_name | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | column_name | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_type | String | FIELD | | No | String | | +| greptime | information_schema | columns | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_comment | String | FIELD | | Yes | String | | +| greptime | information_schema | columns | is_nullable | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_default | String | FIELD | | Yes | String | | +| greptime | information_schema | columns | semantic_type | String | FIELD | | No | String | | +| greptime | information_schema | columns | data_type | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_name | String | FIELD | | No | String | | +| greptime | information_schema | columns | table_name | String | FIELD | | No | String | | +| greptime | information_schema | columns | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | engines | xa | String | FIELD | | No | String | | +| greptime | information_schema | engines | transactions | String | FIELD | | No | String | | +| greptime | information_schema | engines | comment | String | FIELD | | No | String | | +| greptime | information_schema | engines | support | String | FIELD | | No | String | | +| greptime | information_schema | engines | engine | String | FIELD | | No | String | | +| greptime | information_schema | engines | savepoints | String | FIELD | | No | String | | +| greptime | information_schema | events | starts | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | event_comment | String | FIELD | | No | String | | +| greptime | information_schema | events | database_collation | String | FIELD | | No | String | | +| greptime | information_schema | events | collation_connection | String | FIELD | | No | String | | +| greptime | information_schema | events | character_set_client | String | FIELD | | No | String | | +| greptime | information_schema | events | originator | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | events | event_catalog | String | FIELD | | No | String | | +| greptime | information_schema | events | event_schema | String | FIELD | | No | String | | +| greptime | information_schema | events | event_name | String | FIELD | | No | String | | +| greptime | information_schema | events | definer | String | FIELD | | No | String | | +| greptime | information_schema | events | time_zone | String | FIELD | | No | String | | +| greptime | information_schema | events | event_body | String | FIELD | | No | String | | +| greptime | information_schema | events | event_definition | String | FIELD | | No | String | | +| greptime | information_schema | events | event_type | String | FIELD | | No | String | | +| greptime | information_schema | events | execute_at | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | interval_value | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | events | interval_field | String | FIELD | | No | String | | +| greptime | information_schema | events | sql_mode | String | FIELD | | No | String | | +| greptime | information_schema | events | ends | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | status | String | FIELD | | No | String | | +| greptime | information_schema | events | on_completion | String | FIELD | | No | String | | +| greptime | information_schema | events | created | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | last_altered | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | last_executed | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | free_extents | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | row_format | String | FIELD | | No | String | | +| greptime | information_schema | files | extra | String | FIELD | | No | String | | +| greptime | information_schema | files | status | String | FIELD | | No | String | | +| greptime | information_schema | files | checksum | String | FIELD | | No | String | | +| greptime | information_schema | files | check_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | file_id | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | file_name | String | FIELD | | No | String | | +| greptime | information_schema | files | file_type | String | FIELD | | No | String | | +| greptime | information_schema | files | tablespace_name | String | FIELD | | No | String | | +| greptime | information_schema | files | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | files | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | files | table_name | String | FIELD | | No | String | | +| greptime | information_schema | files | logfile_group_name | String | FIELD | | No | String | | +| greptime | information_schema | files | logfile_group_number | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | engine | String | FIELD | | No | String | | +| greptime | information_schema | files | fulltext_keys | String | FIELD | | No | String | | +| greptime | information_schema | files | deleted_rows | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | update_count | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | update_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | total_extents | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | extent_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | initial_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | maximum_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | autoextend_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | creation_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | last_update_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | last_access_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | recover_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | transaction_counter | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | version | String | FIELD | | No | String | | +| greptime | information_schema | files | create_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | table_rows | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | avg_row_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | data_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | max_data_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | index_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | data_free | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | key_column_usage | ordinal_position | UInt32 | FIELD | | No | UInt32 | | +| greptime | information_schema | key_column_usage | constraint_schema | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | referenced_column_name | String | FIELD | | Yes | String | | +| greptime | information_schema | key_column_usage | referenced_table_name | String | FIELD | | Yes | String | | +| greptime | information_schema | key_column_usage | referenced_table_schema | String | FIELD | | Yes | String | | +| greptime | information_schema | key_column_usage | position_in_unique_constraint | UInt32 | FIELD | | Yes | UInt32 | | +| greptime | information_schema | key_column_usage | constraint_catalog | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | column_name | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | constraint_name | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | table_name | String | FIELD | | No | String | | +| greptime | information_schema | schemata | catalog_name | String | FIELD | | No | String | | +| greptime | information_schema | schemata | default_collation_name | String | FIELD | | No | String | | +| greptime | information_schema | schemata | default_character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | schemata | sql_path | String | FIELD | | Yes | String | | +| greptime | information_schema | schemata | schema_name | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | tables | engine | String | FIELD | | Yes | String | | +| greptime | information_schema | tables | table_name | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_type | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_id | UInt32 | FIELD | | Yes | UInt32 | | +| greptime | public | numbers | number | UInt32 | TAG | | No | UInt32 | | ++---------------+--------------------+---------------------------------------+-------------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ create database my_db; @@ -277,6 +290,34 @@ select count(*) from build_info; | 1 | +----------+ +desc table key_column_usage; + ++-------------------------------+--------+-----+------+---------+---------------+ +| Column | Type | Key | Null | Default | Semantic Type | ++-------------------------------+--------+-----+------+---------+---------------+ +| constraint_catalog | String | | NO | | FIELD | +| constraint_schema | String | | NO | | FIELD | +| constraint_name | String | | NO | | FIELD | +| table_catalog | String | | NO | | FIELD | +| table_schema | String | | NO | | FIELD | +| table_name | String | | NO | | FIELD | +| column_name | String | | NO | | FIELD | +| ordinal_position | UInt32 | | NO | | FIELD | +| position_in_unique_constraint | UInt32 | | YES | | FIELD | +| referenced_table_schema | String | | YES | | FIELD | +| referenced_table_name | String | | YES | | FIELD | +| referenced_column_name | String | | YES | | FIELD | ++-------------------------------+--------+-----+------+---------+---------------+ + +select * from key_column_usage; + ++--------------------+-------------------+-----------------+---------------+--------------+------------+-------------+------------------+-------------------------------+-------------------------+-----------------------+------------------------+ +| constraint_catalog | constraint_schema | constraint_name | table_catalog | table_schema | table_name | column_name | ordinal_position | position_in_unique_constraint | referenced_table_schema | referenced_table_name | referenced_column_name | ++--------------------+-------------------+-----------------+---------------+--------------+------------+-------------+------------------+-------------------------------+-------------------------+-----------------------+------------------------+ +| def | my_db | TIME INDEX | def | my_db | foo | ts | 1 | | | | | +| def | public | PRIMARY | def | public | numbers | number | 1 | | | | | ++--------------------+-------------------+-----------------+---------------+--------------+------------+-------------+------------------+-------------------------------+-------------------------+-----------------------+------------------------+ + -- tables not implemented desc table COLUMN_PRIVILEGES; diff --git a/tests/cases/standalone/common/system/information_schema.sql b/tests/cases/standalone/common/system/information_schema.sql index 6550fb544c61..ef0fbdeb7578 100644 --- a/tests/cases/standalone/common/system/information_schema.sql +++ b/tests/cases/standalone/common/system/information_schema.sql @@ -57,6 +57,10 @@ desc table build_info; select count(*) from build_info; +desc table key_column_usage; + +select * from key_column_usage; + -- tables not implemented desc table COLUMN_PRIVILEGES; From c4d7b0d91deeb06098ed54937dc3d39cba8f3769 Mon Sep 17 00:00:00 2001 From: dimbtp Date: Tue, 2 Jan 2024 12:10:59 +0800 Subject: [PATCH 15/21] feat: add some tables for `information_schema` (#3060) * feat: add information_schema.optimizer_trace * feat: add information_schema.parameters * feat: add information_schema.profiling * feat: add information_schema.referential_constraints * feat: add information_schema.routines * feat: add information_schema.schema_privileges * feat: add information_schema.table_privileges * feat: add information_schema.triggers * fix: update sql test result * feat: add information_schema.global_status * feat: add information_schema.session_status * fix: update sql test result * fix: add TODO for some tables * Update src/catalog/src/information_schema/memory_table/tables.rs Co-authored-by: Yingwen --------- Co-authored-by: dennis zhuang Co-authored-by: Yingwen --- src/catalog/src/information_schema.rs | 20 + .../information_schema/memory_table/tables.rs | 184 ++++++++ .../src/information_schema/table_names.rs | 12 +- src/common/catalog/src/consts.rs | 20 + .../common/show/show_databases_tables.result | 10 + .../common/system/information_schema.result | 403 ++++++++++++------ 6 files changed, 510 insertions(+), 139 deletions(-) diff --git a/src/catalog/src/information_schema.rs b/src/catalog/src/information_schema.rs index f806e307b757..7dbc71f6f465 100644 --- a/src/catalog/src/information_schema.rs +++ b/src/catalog/src/information_schema.rs @@ -61,6 +61,16 @@ lazy_static! { CHECK_CONSTRAINTS, EVENTS, FILES, + OPTIMIZER_TRACE, + PARAMETERS, + PROFILING, + REFERENTIAL_CONSTRAINTS, + ROUTINES, + SCHEMA_PRIVILEGES, + TABLE_PRIVILEGES, + TRIGGERS, + GLOBAL_STATUS, + SESSION_STATUS, ]; } @@ -179,6 +189,16 @@ impl InformationSchemaProvider { CHECK_CONSTRAINTS => setup_memory_table!(CHECK_CONSTRAINTS), EVENTS => setup_memory_table!(EVENTS), FILES => setup_memory_table!(FILES), + OPTIMIZER_TRACE => setup_memory_table!(OPTIMIZER_TRACE), + PARAMETERS => setup_memory_table!(PARAMETERS), + PROFILING => setup_memory_table!(PROFILING), + REFERENTIAL_CONSTRAINTS => setup_memory_table!(REFERENTIAL_CONSTRAINTS), + ROUTINES => setup_memory_table!(ROUTINES), + SCHEMA_PRIVILEGES => setup_memory_table!(SCHEMA_PRIVILEGES), + TABLE_PRIVILEGES => setup_memory_table!(TABLE_PRIVILEGES), + TRIGGERS => setup_memory_table!(TRIGGERS), + GLOBAL_STATUS => setup_memory_table!(GLOBAL_STATUS), + SESSION_STATUS => setup_memory_table!(SESSION_STATUS), KEY_COLUMN_USAGE => Some(Arc::new(InformationSchemaKeyColumnUsage::new( self.catalog_name.clone(), self.catalog_manager.clone(), diff --git a/src/catalog/src/information_schema/memory_table/tables.rs b/src/catalog/src/information_schema/memory_table/tables.rs index 30be1fbaa748..9922edd7c49f 100644 --- a/src/catalog/src/information_schema/memory_table/tables.rs +++ b/src/catalog/src/information_schema/memory_table/tables.rs @@ -227,6 +227,190 @@ pub fn get_schema_columns(table_name: &str) -> (SchemaRef, Vec) { vec![], ), + OPTIMIZER_TRACE => ( + vec![ + string_column("QUERY"), + string_column("TRACE"), + bigint_column("MISSING_BYTES_BEYOND_MAX_MEM_SIZE"), + bigint_column("INSUFFICIENT_PRIVILEGES"), + ], + vec![], + ), + + // MySQL(https://dev.mysql.com/doc/refman/8.2/en/information-schema-parameters-table.html) + // has the spec that is different from + // PostgreSQL(https://www.postgresql.org/docs/current/infoschema-parameters.html). + // Follow `MySQL` spec here. + PARAMETERS => ( + vec![ + string_column("SPECIFIC_CATALOG"), + string_column("SPECIFIC_SCHEMA"), + string_column("SPECIFIC_NAME"), + bigint_column("ORDINAL_POSITION"), + string_column("PARAMETER_MODE"), + string_column("PARAMETER_NAME"), + string_column("DATA_TYPE"), + bigint_column("CHARACTER_MAXIMUM_LENGTH"), + bigint_column("CHARACTER_OCTET_LENGTH"), + bigint_column("NUMERIC_PRECISION"), + bigint_column("NUMERIC_SCALE"), + bigint_column("DATETIME_PRECISION"), + string_column("CHARACTER_SET_NAME"), + string_column("COLLATION_NAME"), + string_column("DTD_IDENTIFIER"), + string_column("ROUTINE_TYPE"), + ], + vec![], + ), + + PROFILING => ( + vec![ + bigint_column("QUERY_ID"), + bigint_column("SEQ"), + string_column("STATE"), + bigint_column("DURATION"), + bigint_column("CPU_USER"), + bigint_column("CPU_SYSTEM"), + bigint_column("CONTEXT_VOLUNTARY"), + bigint_column("CONTEXT_INVOLUNTARY"), + bigint_column("BLOCK_OPS_IN"), + bigint_column("BLOCK_OPS_OUT"), + bigint_column("MESSAGES_SENT"), + bigint_column("MESSAGES_RECEIVED"), + bigint_column("PAGE_FAULTS_MAJOR"), + bigint_column("PAGE_FAULTS_MINOR"), + bigint_column("SWAPS"), + string_column("SOURCE_FUNCTION"), + string_column("SOURCE_FILE"), + bigint_column("SOURCE_LINE"), + ], + vec![], + ), + + // TODO: _Must_ reimplement this table when foreign key constraint is supported. + REFERENTIAL_CONSTRAINTS => ( + vec![ + string_column("CONSTRAINT_CATALOG"), + string_column("CONSTRAINT_SCHEMA"), + string_column("CONSTRAINT_NAME"), + string_column("UNIQUE_CONSTRAINT_CATALOG"), + string_column("UNIQUE_CONSTRAINT_SCHEMA"), + string_column("UNIQUE_CONSTRAINT_NAME"), + string_column("MATCH_OPTION"), + string_column("UPDATE_RULE"), + string_column("DELETE_RULE"), + string_column("TABLE_NAME"), + string_column("REFERENCED_TABLE_NAME"), + ], + vec![], + ), + + ROUTINES => ( + vec![ + string_column("SPECIFIC_NAME"), + string_column("ROUTINE_CATALOG"), + string_column("ROUTINE_SCHEMA"), + string_column("ROUTINE_NAME"), + string_column("ROUTINE_TYPE"), + string_column("DATA_TYPE"), + bigint_column("CHARACTER_MAXIMUM_LENGTH"), + bigint_column("CHARACTER_OCTET_LENGTH"), + bigint_column("NUMERIC_PRECISION"), + bigint_column("NUMERIC_SCALE"), + bigint_column("DATETIME_PRECISION"), + string_column("CHARACTER_SET_NAME"), + string_column("COLLATION_NAME"), + string_column("DTD_IDENTIFIER"), + string_column("ROUTINE_BODY"), + string_column("ROUTINE_DEFINITION"), + string_column("EXTERNAL_NAME"), + string_column("EXTERNAL_LANGUAGE"), + string_column("PARAMETER_STYLE"), + string_column("IS_DETERMINISTIC"), + string_column("SQL_DATA_ACCESS"), + string_column("SQL_PATH"), + string_column("SECURITY_TYPE"), + datetime_column("CREATED"), + datetime_column("LAST_ALTERED"), + string_column("SQL_MODE"), + string_column("ROUTINE_COMMENT"), + string_column("DEFINER"), + string_column("CHARACTER_SET_CLIENT"), + string_column("COLLATION_CONNECTION"), + string_column("DATABASE_COLLATION"), + ], + vec![], + ), + + SCHEMA_PRIVILEGES => ( + vec![ + string_column("GRANTEE"), + string_column("TABLE_CATALOG"), + string_column("TABLE_SCHEMA"), + string_column("PRIVILEGE_TYPE"), + string_column("IS_GRANTABLE"), + ], + vec![], + ), + + TABLE_PRIVILEGES => ( + vec![ + string_column("GRANTEE"), + string_column("TABLE_CATALOG"), + string_column("TABLE_SCHEMA"), + string_column("TABLE_NAME"), + string_column("PRIVILEGE_TYPE"), + string_column("IS_GRANTABLE"), + ], + vec![], + ), + + TRIGGERS => ( + vec![ + string_column("TRIGGER_CATALOG"), + string_column("TRIGGER_SCHEMA"), + string_column("TRIGGER_NAME"), + string_column("EVENT_MANIPULATION"), + string_column("EVENT_OBJECT_CATALOG"), + string_column("EVENT_OBJECT_SCHEMA"), + string_column("EVENT_OBJECT_TABLE"), + bigint_column("ACTION_ORDER"), + string_column("ACTION_CONDITION"), + string_column("ACTION_STATEMENT"), + string_column("ACTION_ORIENTATION"), + string_column("ACTION_TIMING"), + string_column("ACTION_REFERENCE_OLD_TABLE"), + string_column("ACTION_REFERENCE_NEW_TABLE"), + string_column("ACTION_REFERENCE_OLD_ROW"), + string_column("ACTION_REFERENCE_NEW_ROW"), + datetime_column("CREATED"), + string_column("SQL_MODE"), + string_column("DEFINER"), + string_column("CHARACTER_SET_CLIENT"), + string_column("COLLATION_CONNECTION"), + string_column("DATABASE_COLLATION"), + ], + vec![], + ), + + // TODO: Considering store internal metrics in `global_status` and + // `session_status` tables. + GLOBAL_STATUS => ( + vec![ + string_column("VARIABLE_NAME"), + string_column("VARIABLE_VALUE"), + ], + vec![], + ), + + SESSION_STATUS => ( + vec![ + string_column("VARIABLE_NAME"), + string_column("VARIABLE_VALUE"), + ], + vec![], + ), + _ => unreachable!("Unknown table in information_schema: {}", table_name), }; diff --git a/src/catalog/src/information_schema/table_names.rs b/src/catalog/src/information_schema/table_names.rs index e1b23b4f6938..ce252e2e47dd 100644 --- a/src/catalog/src/information_schema/table_names.rs +++ b/src/catalog/src/information_schema/table_names.rs @@ -25,6 +25,16 @@ pub const COLLATIONS: &str = "collations"; pub const COLLATION_CHARACTER_SET_APPLICABILITY: &str = "collation_character_set_applicability"; pub const CHECK_CONSTRAINTS: &str = "check_constraints"; pub const EVENTS: &str = "events"; -pub const KEY_COLUMN_USAGE: &str = "key_column_usage"; pub const FILES: &str = "files"; pub const SCHEMATA: &str = "schemata"; +pub const KEY_COLUMN_USAGE: &str = "key_column_usage"; +pub const OPTIMIZER_TRACE: &str = "optimizer_trace"; +pub const PARAMETERS: &str = "parameters"; +pub const PROFILING: &str = "profiling"; +pub const REFERENTIAL_CONSTRAINTS: &str = "referential_constraints"; +pub const ROUTINES: &str = "routines"; +pub const SCHEMA_PRIVILEGES: &str = "schema_privileges"; +pub const TABLE_PRIVILEGES: &str = "table_privileges"; +pub const TRIGGERS: &str = "triggers"; +pub const GLOBAL_STATUS: &str = "global_status"; +pub const SESSION_STATUS: &str = "session_status"; diff --git a/src/common/catalog/src/consts.rs b/src/common/catalog/src/consts.rs index 6af76120a5f8..3e0510cd9215 100644 --- a/src/common/catalog/src/consts.rs +++ b/src/common/catalog/src/consts.rs @@ -60,6 +60,26 @@ pub const INFORMATION_SCHEMA_FILES_TABLE_ID: u32 = 14; pub const INFORMATION_SCHEMA_SCHEMATA_TABLE_ID: u32 = 15; /// id for information_schema.KEY_COLUMN_USAGE pub const INFORMATION_SCHEMA_KEY_COLUMN_USAGE_TABLE_ID: u32 = 16; +/// id for information_schema.OPTIMIZER_TRACE +pub const INFORMATION_SCHEMA_OPTIMIZER_TRACE_TABLE_ID: u32 = 17; +/// id for information_schema.PARAMETERS +pub const INFORMATION_SCHEMA_PARAMETERS_TABLE_ID: u32 = 18; +/// id for information_schema.PROFILING +pub const INFORMATION_SCHEMA_PROFILING_TABLE_ID: u32 = 19; +/// id for information_schema.REFERENTIAL_CONSTRAINTS +pub const INFORMATION_SCHEMA_REFERENTIAL_CONSTRAINTS_TABLE_ID: u32 = 20; +/// id for information_schema.ROUTINES +pub const INFORMATION_SCHEMA_ROUTINES_TABLE_ID: u32 = 21; +/// id for information_schema.SCHEMA_PRIVILEGES +pub const INFORMATION_SCHEMA_SCHEMA_PRIVILEGES_TABLE_ID: u32 = 22; +/// id for information_schema.TABLE_PRIVILEGES +pub const INFORMATION_SCHEMA_TABLE_PRIVILEGES_TABLE_ID: u32 = 23; +/// id for information_schema.TRIGGERS +pub const INFORMATION_SCHEMA_TRIGGERS_TABLE_ID: u32 = 24; +/// id for information_schema.GLOBAL_STATUS +pub const INFORMATION_SCHEMA_GLOBAL_STATUS_TABLE_ID: u32 = 25; +/// id for information_schema.SESSION_STATUS +pub const INFORMATION_SCHEMA_SESSION_STATUS_TABLE_ID: u32 = 26; /// ----- End of information_schema tables ----- pub const MITO_ENGINE: &str = "mito"; diff --git a/tests/cases/standalone/common/show/show_databases_tables.result b/tests/cases/standalone/common/show/show_databases_tables.result index 76a69d56aa8b..0f5615222726 100644 --- a/tests/cases/standalone/common/show/show_databases_tables.result +++ b/tests/cases/standalone/common/show/show_databases_tables.result @@ -31,8 +31,18 @@ show tables; | engines | | events | | files | +| global_status | | key_column_usage | +| optimizer_trace | +| parameters | +| profiling | +| referential_constraints | +| routines | +| schema_privileges | | schemata | +| session_status | +| table_privileges | | tables | +| triggers | +---------------------------------------+ diff --git a/tests/cases/standalone/common/system/information_schema.result b/tests/cases/standalone/common/system/information_schema.result index a119b028a344..2243265bf3a9 100644 --- a/tests/cases/standalone/common/system/information_schema.result +++ b/tests/cases/standalone/common/system/information_schema.result @@ -23,152 +23,279 @@ order by table_schema, table_name; | greptime | information_schema | engines | LOCAL TEMPORARY | 5 | | | greptime | information_schema | events | LOCAL TEMPORARY | 13 | | | greptime | information_schema | files | LOCAL TEMPORARY | 14 | | +| greptime | information_schema | global_status | LOCAL TEMPORARY | 25 | | | greptime | information_schema | key_column_usage | LOCAL TEMPORARY | 16 | | +| greptime | information_schema | optimizer_trace | LOCAL TEMPORARY | 17 | | +| greptime | information_schema | parameters | LOCAL TEMPORARY | 18 | | +| greptime | information_schema | profiling | LOCAL TEMPORARY | 19 | | +| greptime | information_schema | referential_constraints | LOCAL TEMPORARY | 20 | | +| greptime | information_schema | routines | LOCAL TEMPORARY | 21 | | +| greptime | information_schema | schema_privileges | LOCAL TEMPORARY | 22 | | | greptime | information_schema | schemata | LOCAL TEMPORARY | 15 | | +| greptime | information_schema | session_status | LOCAL TEMPORARY | 26 | | +| greptime | information_schema | table_privileges | LOCAL TEMPORARY | 23 | | | greptime | information_schema | tables | LOCAL TEMPORARY | 3 | | +| greptime | information_schema | triggers | LOCAL TEMPORARY | 24 | | | greptime | public | numbers | LOCAL TEMPORARY | 2 | test_engine | +---------------+--------------------+---------------------------------------+-----------------+----------+-------------+ select * from information_schema.columns order by table_schema, table_name; -+---------------+--------------------+---------------------------------------+-------------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ -| table_catalog | table_schema | table_name | column_name | data_type | semantic_type | column_default | is_nullable | column_type | column_comment | -+---------------+--------------------+---------------------------------------+-------------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ -| greptime | information_schema | build_info | git_branch | String | FIELD | | No | String | | -| greptime | information_schema | build_info | git_commit | String | FIELD | | No | String | | -| greptime | information_schema | build_info | git_commit_short | String | FIELD | | No | String | | -| greptime | information_schema | build_info | git_dirty | String | FIELD | | No | String | | -| greptime | information_schema | build_info | pkg_version | String | FIELD | | No | String | | -| greptime | information_schema | character_sets | description | String | FIELD | | No | String | | -| greptime | information_schema | character_sets | character_set_name | String | FIELD | | No | String | | -| greptime | information_schema | character_sets | default_collate_name | String | FIELD | | No | String | | -| greptime | information_schema | character_sets | maxlen | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | check_constraints | check_clause | String | FIELD | | No | String | | -| greptime | information_schema | check_constraints | constraint_name | String | FIELD | | No | String | | -| greptime | information_schema | check_constraints | constraint_schema | String | FIELD | | No | String | | -| greptime | information_schema | check_constraints | constraint_catalog | String | FIELD | | No | String | | -| greptime | information_schema | collation_character_set_applicability | character_set_name | String | FIELD | | No | String | | -| greptime | information_schema | collation_character_set_applicability | collation_name | String | FIELD | | No | String | | -| greptime | information_schema | collations | collation_name | String | FIELD | | No | String | | -| greptime | information_schema | collations | character_set_name | String | FIELD | | No | String | | -| greptime | information_schema | collations | id | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | collations | is_default | String | FIELD | | No | String | | -| greptime | information_schema | collations | is_compiled | String | FIELD | | No | String | | -| greptime | information_schema | collations | sortlen | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | column_privileges | table_schema | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | grantee | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | table_catalog | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | table_name | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | column_name | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | privilege_type | String | FIELD | | No | String | | -| greptime | information_schema | column_privileges | is_grantable | String | FIELD | | No | String | | -| greptime | information_schema | column_statistics | histogram | String | FIELD | | No | String | | -| greptime | information_schema | column_statistics | schema_name | String | FIELD | | No | String | | -| greptime | information_schema | column_statistics | table_name | String | FIELD | | No | String | | -| greptime | information_schema | column_statistics | column_name | String | FIELD | | No | String | | -| greptime | information_schema | columns | column_type | String | FIELD | | No | String | | -| greptime | information_schema | columns | table_catalog | String | FIELD | | No | String | | -| greptime | information_schema | columns | column_comment | String | FIELD | | Yes | String | | -| greptime | information_schema | columns | is_nullable | String | FIELD | | No | String | | -| greptime | information_schema | columns | column_default | String | FIELD | | Yes | String | | -| greptime | information_schema | columns | semantic_type | String | FIELD | | No | String | | -| greptime | information_schema | columns | data_type | String | FIELD | | No | String | | -| greptime | information_schema | columns | column_name | String | FIELD | | No | String | | -| greptime | information_schema | columns | table_name | String | FIELD | | No | String | | -| greptime | information_schema | columns | table_schema | String | FIELD | | No | String | | -| greptime | information_schema | engines | xa | String | FIELD | | No | String | | -| greptime | information_schema | engines | transactions | String | FIELD | | No | String | | -| greptime | information_schema | engines | comment | String | FIELD | | No | String | | -| greptime | information_schema | engines | support | String | FIELD | | No | String | | -| greptime | information_schema | engines | engine | String | FIELD | | No | String | | -| greptime | information_schema | engines | savepoints | String | FIELD | | No | String | | -| greptime | information_schema | events | starts | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | events | event_comment | String | FIELD | | No | String | | -| greptime | information_schema | events | database_collation | String | FIELD | | No | String | | -| greptime | information_schema | events | collation_connection | String | FIELD | | No | String | | -| greptime | information_schema | events | character_set_client | String | FIELD | | No | String | | -| greptime | information_schema | events | originator | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | events | event_catalog | String | FIELD | | No | String | | -| greptime | information_schema | events | event_schema | String | FIELD | | No | String | | -| greptime | information_schema | events | event_name | String | FIELD | | No | String | | -| greptime | information_schema | events | definer | String | FIELD | | No | String | | -| greptime | information_schema | events | time_zone | String | FIELD | | No | String | | -| greptime | information_schema | events | event_body | String | FIELD | | No | String | | -| greptime | information_schema | events | event_definition | String | FIELD | | No | String | | -| greptime | information_schema | events | event_type | String | FIELD | | No | String | | -| greptime | information_schema | events | execute_at | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | events | interval_value | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | events | interval_field | String | FIELD | | No | String | | -| greptime | information_schema | events | sql_mode | String | FIELD | | No | String | | -| greptime | information_schema | events | ends | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | events | status | String | FIELD | | No | String | | -| greptime | information_schema | events | on_completion | String | FIELD | | No | String | | -| greptime | information_schema | events | created | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | events | last_altered | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | events | last_executed | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | free_extents | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | row_format | String | FIELD | | No | String | | -| greptime | information_schema | files | extra | String | FIELD | | No | String | | -| greptime | information_schema | files | status | String | FIELD | | No | String | | -| greptime | information_schema | files | checksum | String | FIELD | | No | String | | -| greptime | information_schema | files | check_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | file_id | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | file_name | String | FIELD | | No | String | | -| greptime | information_schema | files | file_type | String | FIELD | | No | String | | -| greptime | information_schema | files | tablespace_name | String | FIELD | | No | String | | -| greptime | information_schema | files | table_catalog | String | FIELD | | No | String | | -| greptime | information_schema | files | table_schema | String | FIELD | | No | String | | -| greptime | information_schema | files | table_name | String | FIELD | | No | String | | -| greptime | information_schema | files | logfile_group_name | String | FIELD | | No | String | | -| greptime | information_schema | files | logfile_group_number | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | engine | String | FIELD | | No | String | | -| greptime | information_schema | files | fulltext_keys | String | FIELD | | No | String | | -| greptime | information_schema | files | deleted_rows | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | update_count | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | update_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | total_extents | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | extent_size | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | initial_size | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | maximum_size | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | autoextend_size | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | creation_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | last_update_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | last_access_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | recover_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | transaction_counter | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | version | String | FIELD | | No | String | | -| greptime | information_schema | files | create_time | DateTime | FIELD | | No | DateTime | | -| greptime | information_schema | files | table_rows | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | avg_row_length | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | data_length | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | max_data_length | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | index_length | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | files | data_free | Int64 | FIELD | | No | Int64 | | -| greptime | information_schema | key_column_usage | ordinal_position | UInt32 | FIELD | | No | UInt32 | | -| greptime | information_schema | key_column_usage | constraint_schema | String | FIELD | | No | String | | -| greptime | information_schema | key_column_usage | referenced_column_name | String | FIELD | | Yes | String | | -| greptime | information_schema | key_column_usage | referenced_table_name | String | FIELD | | Yes | String | | -| greptime | information_schema | key_column_usage | referenced_table_schema | String | FIELD | | Yes | String | | -| greptime | information_schema | key_column_usage | position_in_unique_constraint | UInt32 | FIELD | | Yes | UInt32 | | -| greptime | information_schema | key_column_usage | constraint_catalog | String | FIELD | | No | String | | -| greptime | information_schema | key_column_usage | column_name | String | FIELD | | No | String | | -| greptime | information_schema | key_column_usage | table_schema | String | FIELD | | No | String | | -| greptime | information_schema | key_column_usage | table_catalog | String | FIELD | | No | String | | -| greptime | information_schema | key_column_usage | constraint_name | String | FIELD | | No | String | | -| greptime | information_schema | key_column_usage | table_name | String | FIELD | | No | String | | -| greptime | information_schema | schemata | catalog_name | String | FIELD | | No | String | | -| greptime | information_schema | schemata | default_collation_name | String | FIELD | | No | String | | -| greptime | information_schema | schemata | default_character_set_name | String | FIELD | | No | String | | -| greptime | information_schema | schemata | sql_path | String | FIELD | | Yes | String | | -| greptime | information_schema | schemata | schema_name | String | FIELD | | No | String | | -| greptime | information_schema | tables | table_schema | String | FIELD | | No | String | | -| greptime | information_schema | tables | table_catalog | String | FIELD | | No | String | | -| greptime | information_schema | tables | engine | String | FIELD | | Yes | String | | -| greptime | information_schema | tables | table_name | String | FIELD | | No | String | | -| greptime | information_schema | tables | table_type | String | FIELD | | No | String | | -| greptime | information_schema | tables | table_id | UInt32 | FIELD | | Yes | UInt32 | | -| greptime | public | numbers | number | UInt32 | TAG | | No | UInt32 | | -+---------------+--------------------+---------------------------------------+-------------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ ++---------------+--------------------+---------------------------------------+-----------------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ +| table_catalog | table_schema | table_name | column_name | data_type | semantic_type | column_default | is_nullable | column_type | column_comment | ++---------------+--------------------+---------------------------------------+-----------------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ +| greptime | information_schema | build_info | git_branch | String | FIELD | | No | String | | +| greptime | information_schema | build_info | git_commit | String | FIELD | | No | String | | +| greptime | information_schema | build_info | git_commit_short | String | FIELD | | No | String | | +| greptime | information_schema | build_info | git_dirty | String | FIELD | | No | String | | +| greptime | information_schema | build_info | pkg_version | String | FIELD | | No | String | | +| greptime | information_schema | character_sets | maxlen | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | character_sets | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | character_sets | default_collate_name | String | FIELD | | No | String | | +| greptime | information_schema | character_sets | description | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | check_clause | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | constraint_catalog | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | constraint_schema | String | FIELD | | No | String | | +| greptime | information_schema | check_constraints | constraint_name | String | FIELD | | No | String | | +| greptime | information_schema | collation_character_set_applicability | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | collation_character_set_applicability | collation_name | String | FIELD | | No | String | | +| greptime | information_schema | collations | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | collations | collation_name | String | FIELD | | No | String | | +| greptime | information_schema | collations | sortlen | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | collations | id | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | collations | is_default | String | FIELD | | No | String | | +| greptime | information_schema | collations | is_compiled | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | grantee | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | table_name | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | column_name | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | privilege_type | String | FIELD | | No | String | | +| greptime | information_schema | column_privileges | is_grantable | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | histogram | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | schema_name | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | table_name | String | FIELD | | No | String | | +| greptime | information_schema | column_statistics | column_name | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_type | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_comment | String | FIELD | | Yes | String | | +| greptime | information_schema | columns | table_name | String | FIELD | | No | String | | +| greptime | information_schema | columns | is_nullable | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_default | String | FIELD | | Yes | String | | +| greptime | information_schema | columns | semantic_type | String | FIELD | | No | String | | +| greptime | information_schema | columns | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | columns | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | columns | data_type | String | FIELD | | No | String | | +| greptime | information_schema | columns | column_name | String | FIELD | | No | String | | +| greptime | information_schema | engines | savepoints | String | FIELD | | No | String | | +| greptime | information_schema | engines | xa | String | FIELD | | No | String | | +| greptime | information_schema | engines | transactions | String | FIELD | | No | String | | +| greptime | information_schema | engines | comment | String | FIELD | | No | String | | +| greptime | information_schema | engines | support | String | FIELD | | No | String | | +| greptime | information_schema | engines | engine | String | FIELD | | No | String | | +| greptime | information_schema | events | event_name | String | FIELD | | No | String | | +| greptime | information_schema | events | last_executed | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | database_collation | String | FIELD | | No | String | | +| greptime | information_schema | events | collation_connection | String | FIELD | | No | String | | +| greptime | information_schema | events | character_set_client | String | FIELD | | No | String | | +| greptime | information_schema | events | originator | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | events | event_catalog | String | FIELD | | No | String | | +| greptime | information_schema | events | event_schema | String | FIELD | | No | String | | +| greptime | information_schema | events | event_comment | String | FIELD | | No | String | | +| greptime | information_schema | events | definer | String | FIELD | | No | String | | +| greptime | information_schema | events | time_zone | String | FIELD | | No | String | | +| greptime | information_schema | events | event_body | String | FIELD | | No | String | | +| greptime | information_schema | events | event_definition | String | FIELD | | No | String | | +| greptime | information_schema | events | event_type | String | FIELD | | No | String | | +| greptime | information_schema | events | execute_at | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | interval_value | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | events | interval_field | String | FIELD | | No | String | | +| greptime | information_schema | events | sql_mode | String | FIELD | | No | String | | +| greptime | information_schema | events | starts | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | ends | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | status | String | FIELD | | No | String | | +| greptime | information_schema | events | on_completion | String | FIELD | | No | String | | +| greptime | information_schema | events | created | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | events | last_altered | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | file_name | String | FIELD | | No | String | | +| greptime | information_schema | files | free_extents | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | checksum | String | FIELD | | No | String | | +| greptime | information_schema | files | update_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | create_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | status | String | FIELD | | No | String | | +| greptime | information_schema | files | file_id | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | check_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | file_type | String | FIELD | | No | String | | +| greptime | information_schema | files | tablespace_name | String | FIELD | | No | String | | +| greptime | information_schema | files | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | files | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | files | table_name | String | FIELD | | No | String | | +| greptime | information_schema | files | logfile_group_name | String | FIELD | | No | String | | +| greptime | information_schema | files | logfile_group_number | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | engine | String | FIELD | | No | String | | +| greptime | information_schema | files | fulltext_keys | String | FIELD | | No | String | | +| greptime | information_schema | files | deleted_rows | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | update_count | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | extra | String | FIELD | | No | String | | +| greptime | information_schema | files | total_extents | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | extent_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | initial_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | maximum_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | autoextend_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | creation_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | last_update_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | last_access_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | recover_time | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | files | transaction_counter | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | version | String | FIELD | | No | String | | +| greptime | information_schema | files | row_format | String | FIELD | | No | String | | +| greptime | information_schema | files | table_rows | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | avg_row_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | data_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | max_data_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | index_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | files | data_free | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | global_status | variable_name | String | FIELD | | No | String | | +| greptime | information_schema | global_status | variable_value | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | ordinal_position | UInt32 | FIELD | | No | UInt32 | | +| greptime | information_schema | key_column_usage | referenced_column_name | String | FIELD | | Yes | String | | +| greptime | information_schema | key_column_usage | referenced_table_name | String | FIELD | | Yes | String | | +| greptime | information_schema | key_column_usage | referenced_table_schema | String | FIELD | | Yes | String | | +| greptime | information_schema | key_column_usage | position_in_unique_constraint | UInt32 | FIELD | | Yes | UInt32 | | +| greptime | information_schema | key_column_usage | constraint_catalog | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | constraint_schema | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | constraint_name | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | table_name | String | FIELD | | No | String | | +| greptime | information_schema | key_column_usage | column_name | String | FIELD | | No | String | | +| greptime | information_schema | optimizer_trace | insufficient_privileges | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | optimizer_trace | missing_bytes_beyond_max_mem_size | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | optimizer_trace | trace | String | FIELD | | No | String | | +| greptime | information_schema | optimizer_trace | query | String | FIELD | | No | String | | +| greptime | information_schema | parameters | specific_catalog | String | FIELD | | No | String | | +| greptime | information_schema | parameters | character_maximum_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | parameters | routine_type | String | FIELD | | No | String | | +| greptime | information_schema | parameters | dtd_identifier | String | FIELD | | No | String | | +| greptime | information_schema | parameters | collation_name | String | FIELD | | No | String | | +| greptime | information_schema | parameters | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | parameters | datetime_precision | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | parameters | numeric_scale | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | parameters | numeric_precision | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | parameters | character_octet_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | parameters | specific_schema | String | FIELD | | No | String | | +| greptime | information_schema | parameters | specific_name | String | FIELD | | No | String | | +| greptime | information_schema | parameters | ordinal_position | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | parameters | parameter_mode | String | FIELD | | No | String | | +| greptime | information_schema | parameters | parameter_name | String | FIELD | | No | String | | +| greptime | information_schema | parameters | data_type | String | FIELD | | No | String | | +| greptime | information_schema | profiling | block_ops_out | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | messages_sent | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | source_line | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | source_file | String | FIELD | | No | String | | +| greptime | information_schema | profiling | source_function | String | FIELD | | No | String | | +| greptime | information_schema | profiling | swaps | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | page_faults_minor | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | page_faults_major | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | messages_received | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | query_id | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | seq | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | state | String | FIELD | | No | String | | +| greptime | information_schema | profiling | duration | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | cpu_user | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | cpu_system | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | context_voluntary | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | context_involuntary | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | profiling | block_ops_in | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | referential_constraints | referenced_table_name | String | FIELD | | No | String | | +| greptime | information_schema | referential_constraints | constraint_name | String | FIELD | | No | String | | +| greptime | information_schema | referential_constraints | table_name | String | FIELD | | No | String | | +| greptime | information_schema | referential_constraints | delete_rule | String | FIELD | | No | String | | +| greptime | information_schema | referential_constraints | update_rule | String | FIELD | | No | String | | +| greptime | information_schema | referential_constraints | match_option | String | FIELD | | No | String | | +| greptime | information_schema | referential_constraints | unique_constraint_name | String | FIELD | | No | String | | +| greptime | information_schema | referential_constraints | unique_constraint_schema | String | FIELD | | No | String | | +| greptime | information_schema | referential_constraints | unique_constraint_catalog | String | FIELD | | No | String | | +| greptime | information_schema | referential_constraints | constraint_catalog | String | FIELD | | No | String | | +| greptime | information_schema | referential_constraints | constraint_schema | String | FIELD | | No | String | | +| greptime | information_schema | routines | sql_mode | String | FIELD | | No | String | | +| greptime | information_schema | routines | security_type | String | FIELD | | No | String | | +| greptime | information_schema | routines | database_collation | String | FIELD | | No | String | | +| greptime | information_schema | routines | data_type | String | FIELD | | No | String | | +| greptime | information_schema | routines | character_set_client | String | FIELD | | No | String | | +| greptime | information_schema | routines | definer | String | FIELD | | No | String | | +| greptime | information_schema | routines | routine_comment | String | FIELD | | No | String | | +| greptime | information_schema | routines | last_altered | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | routines | created | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | routines | specific_name | String | FIELD | | No | String | | +| greptime | information_schema | routines | routine_catalog | String | FIELD | | No | String | | +| greptime | information_schema | routines | routine_schema | String | FIELD | | No | String | | +| greptime | information_schema | routines | routine_name | String | FIELD | | No | String | | +| greptime | information_schema | routines | routine_type | String | FIELD | | No | String | | +| greptime | information_schema | routines | collation_connection | String | FIELD | | No | String | | +| greptime | information_schema | routines | character_maximum_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | routines | character_octet_length | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | routines | numeric_precision | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | routines | numeric_scale | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | routines | datetime_precision | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | routines | character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | routines | collation_name | String | FIELD | | No | String | | +| greptime | information_schema | routines | dtd_identifier | String | FIELD | | No | String | | +| greptime | information_schema | routines | routine_body | String | FIELD | | No | String | | +| greptime | information_schema | routines | routine_definition | String | FIELD | | No | String | | +| greptime | information_schema | routines | external_name | String | FIELD | | No | String | | +| greptime | information_schema | routines | external_language | String | FIELD | | No | String | | +| greptime | information_schema | routines | parameter_style | String | FIELD | | No | String | | +| greptime | information_schema | routines | is_deterministic | String | FIELD | | No | String | | +| greptime | information_schema | routines | sql_data_access | String | FIELD | | No | String | | +| greptime | information_schema | routines | sql_path | String | FIELD | | No | String | | +| greptime | information_schema | schema_privileges | is_grantable | String | FIELD | | No | String | | +| greptime | information_schema | schema_privileges | grantee | String | FIELD | | No | String | | +| greptime | information_schema | schema_privileges | privilege_type | String | FIELD | | No | String | | +| greptime | information_schema | schema_privileges | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | schema_privileges | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | schemata | schema_name | String | FIELD | | No | String | | +| greptime | information_schema | schemata | catalog_name | String | FIELD | | No | String | | +| greptime | information_schema | schemata | sql_path | String | FIELD | | Yes | String | | +| greptime | information_schema | schemata | default_character_set_name | String | FIELD | | No | String | | +| greptime | information_schema | schemata | default_collation_name | String | FIELD | | No | String | | +| greptime | information_schema | session_status | variable_value | String | FIELD | | No | String | | +| greptime | information_schema | session_status | variable_name | String | FIELD | | No | String | | +| greptime | information_schema | table_privileges | grantee | String | FIELD | | No | String | | +| greptime | information_schema | table_privileges | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | table_privileges | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | table_privileges | table_name | String | FIELD | | No | String | | +| greptime | information_schema | table_privileges | privilege_type | String | FIELD | | No | String | | +| greptime | information_schema | table_privileges | is_grantable | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_catalog | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_name | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_type | String | FIELD | | No | String | | +| greptime | information_schema | tables | engine | String | FIELD | | Yes | String | | +| greptime | information_schema | tables | table_schema | String | FIELD | | No | String | | +| greptime | information_schema | tables | table_id | UInt32 | FIELD | | Yes | UInt32 | | +| greptime | information_schema | triggers | event_manipulation | String | FIELD | | No | String | | +| greptime | information_schema | triggers | event_object_schema | String | FIELD | | No | String | | +| greptime | information_schema | triggers | event_object_table | String | FIELD | | No | String | | +| greptime | information_schema | triggers | action_order | Int64 | FIELD | | No | Int64 | | +| greptime | information_schema | triggers | action_condition | String | FIELD | | No | String | | +| greptime | information_schema | triggers | action_statement | String | FIELD | | No | String | | +| greptime | information_schema | triggers | action_orientation | String | FIELD | | No | String | | +| greptime | information_schema | triggers | action_timing | String | FIELD | | No | String | | +| greptime | information_schema | triggers | action_reference_old_table | String | FIELD | | No | String | | +| greptime | information_schema | triggers | action_reference_new_table | String | FIELD | | No | String | | +| greptime | information_schema | triggers | action_reference_old_row | String | FIELD | | No | String | | +| greptime | information_schema | triggers | action_reference_new_row | String | FIELD | | No | String | | +| greptime | information_schema | triggers | created | DateTime | FIELD | | No | DateTime | | +| greptime | information_schema | triggers | sql_mode | String | FIELD | | No | String | | +| greptime | information_schema | triggers | definer | String | FIELD | | No | String | | +| greptime | information_schema | triggers | character_set_client | String | FIELD | | No | String | | +| greptime | information_schema | triggers | collation_connection | String | FIELD | | No | String | | +| greptime | information_schema | triggers | database_collation | String | FIELD | | No | String | | +| greptime | information_schema | triggers | trigger_catalog | String | FIELD | | No | String | | +| greptime | information_schema | triggers | event_object_catalog | String | FIELD | | No | String | | +| greptime | information_schema | triggers | trigger_name | String | FIELD | | No | String | | +| greptime | information_schema | triggers | trigger_schema | String | FIELD | | No | String | | +| greptime | public | numbers | number | UInt32 | TAG | | No | UInt32 | | ++---------------+--------------------+---------------------------------------+-----------------------------------+-----------+---------------+----------------+-------------+-------------+----------------+ create database my_db; From 5653389063be730f4fd50420a5de412ddbac8222 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Tue, 2 Jan 2024 16:31:37 +0900 Subject: [PATCH 16/21] feat!: correct the kafka config option (#3065) * feat: correct the kafka config option * refactor: rewrite the verbose comments --- config/datanode.example.toml | 2 +- config/standalone.example.toml | 20 ++++++++++---------- src/common/config/src/wal.rs | 4 ++-- src/common/config/src/wal/kafka.rs | 18 ++++++++---------- src/common/meta/src/wal/kafka.rs | 2 +- src/log-store/src/kafka/log_store.rs | 2 +- 6 files changed, 23 insertions(+), 25 deletions(-) diff --git a/config/datanode.example.toml b/config/datanode.example.toml index bd3f8fc2eec9..8f81829eab40 100644 --- a/config/datanode.example.toml +++ b/config/datanode.example.toml @@ -53,7 +53,7 @@ sync_write = false # broker_endpoints = ["127.0.0.1:9092"] # max_batch_size = "4MB" # linger = "200ms" -# produce_record_timeout = "100ms" +# consumer_wait_timeout = "100ms" # backoff_init = "500ms" # backoff_max = "10s" # backoff_base = 2 diff --git a/config/standalone.example.toml b/config/standalone.example.toml index 0fa58dd413c6..cb2c0bd61ab9 100644 --- a/config/standalone.example.toml +++ b/config/standalone.example.toml @@ -100,29 +100,29 @@ provider = "raft_engine" # Available selector types: # - "round_robin" (default) # selector_type = "round_robin" -# A Kafka topic is constructed by concatenating `topic_name_prefix` and `topic_id`. +# The prefix of topic name. # topic_name_prefix = "greptimedb_wal_topic" # Number of partitions per topic. # num_partitions = 1 -# Expected number of replicas of each partition. +# The number of replicas of each partition. # replication_factor = 1 -# The maximum log size a kafka batch producer could buffer. +# The max size of a single producer batch. # max_batch_size = "4MB" -# The linger duration of a kafka batch producer. +# The linger duration. # linger = "200ms" -# The maximum amount of time (in milliseconds) to wait for Kafka records to be returned. -# produce_record_timeout = "100ms" -# Above which a topic creation operation will be cancelled. +# The consumer wait timeout. +# consumer_wait_timeout = "100ms" +# Create topic timeout. # create_topic_timeout = "30s" -# The initial backoff for kafka clients. +# The initial backoff delay. # backoff_init = "500ms" -# The maximum backoff for kafka clients. +# The maximum backoff delay. # backoff_max = "10s" # Exponential backoff rate, i.e. next backoff = base * current backoff. # backoff_base = 2 -# Stop reconnecting if the total wait time reaches the deadline. If this config is missing, the reconnecting won't terminate. +# The deadline of retries. # backoff_deadline = "5mins" # WAL data directory diff --git a/src/common/config/src/wal.rs b/src/common/config/src/wal.rs index f9c492758e63..6f0a9867804c 100644 --- a/src/common/config/src/wal.rs +++ b/src/common/config/src/wal.rs @@ -94,7 +94,7 @@ mod tests { broker_endpoints = ["127.0.0.1:9092"] max_batch_size = "4MB" linger = "200ms" - produce_record_timeout = "100ms" + consumer_wait_timeout = "100ms" backoff_init = "500ms" backoff_max = "10s" backoff_base = 2 @@ -106,7 +106,7 @@ mod tests { compression: RsKafkaCompression::default(), max_batch_size: ReadableSize::mb(4), linger: Duration::from_millis(200), - produce_record_timeout: Duration::from_millis(100), + consumer_wait_timeout: Duration::from_millis(100), backoff: KafkaBackoffConfig { init: Duration::from_millis(500), max: Duration::from_secs(10), diff --git a/src/common/config/src/wal/kafka.rs b/src/common/config/src/wal/kafka.rs index 858991264bb6..d510e973451c 100644 --- a/src/common/config/src/wal/kafka.rs +++ b/src/common/config/src/wal/kafka.rs @@ -42,14 +42,14 @@ pub struct KafkaConfig { #[serde(skip)] #[serde(default)] pub compression: RsKafkaCompression, - /// The maximum log size a kafka batch producer could buffer. + /// The max size of a single producer batch. pub max_batch_size: ReadableSize, /// The linger duration of a kafka batch producer. #[serde(with = "humantime_serde")] pub linger: Duration, - /// The maximum amount of time (in milliseconds) to wait for Kafka records to be returned. + /// The consumer wait timeout. #[serde(with = "humantime_serde")] - pub produce_record_timeout: Duration, + pub consumer_wait_timeout: Duration, /// The backoff config. #[serde(flatten, with = "kafka_backoff")] pub backoff: KafkaBackoffConfig, @@ -62,7 +62,7 @@ impl Default for KafkaConfig { compression: RsKafkaCompression::NoCompression, max_batch_size: ReadableSize::mb(4), linger: Duration::from_millis(200), - produce_record_timeout: Duration::from_millis(100), + consumer_wait_timeout: Duration::from_millis(100), backoff: KafkaBackoffConfig::default(), } } @@ -73,17 +73,15 @@ with_prefix!(pub kafka_backoff "backoff_"); #[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] #[serde(default)] pub struct KafkaBackoffConfig { - /// The initial backoff for kafka clients. + /// The initial backoff delay. #[serde(with = "humantime_serde")] pub init: Duration, - /// The maximum backoff for kafka clients. + /// The maximum backoff delay. #[serde(with = "humantime_serde")] pub max: Duration, /// Exponential backoff rate, i.e. next backoff = base * current backoff. - // Sets to u32 type since some structs containing the KafkaConfig need to derive the Eq trait. pub base: u32, - /// Stop reconnecting if the total wait time reaches the deadline. - /// If it's None, the reconnecting won't terminate. + /// The deadline of retries. `None` stands for no deadline. #[serde(with = "humantime_serde")] pub deadline: Option, } @@ -114,7 +112,7 @@ pub struct StandaloneKafkaConfig { pub num_partitions: i32, /// The replication factor of each topic. pub replication_factor: i16, - /// Above which a topic creation operation will be cancelled. + /// The timeout of topic creation. #[serde(with = "humantime_serde")] pub create_topic_timeout: Duration, } diff --git a/src/common/meta/src/wal/kafka.rs b/src/common/meta/src/wal/kafka.rs index 6719f2f63849..703dfa7e3de0 100644 --- a/src/common/meta/src/wal/kafka.rs +++ b/src/common/meta/src/wal/kafka.rs @@ -41,7 +41,7 @@ pub struct KafkaConfig { pub num_partitions: i32, /// The replication factor of each topic. pub replication_factor: i16, - /// Above which a topic creation operation will be cancelled. + /// The timeout of topic creation. #[serde(with = "humantime_serde")] pub create_topic_timeout: Duration, /// The backoff config. diff --git a/src/log-store/src/kafka/log_store.rs b/src/log-store/src/kafka/log_store.rs index 20bcd4e7cf50..2e5543341506 100644 --- a/src/log-store/src/kafka/log_store.rs +++ b/src/log-store/src/kafka/log_store.rs @@ -162,7 +162,7 @@ impl LogStore for KafkaLogStore { let mut stream_consumer = StreamConsumerBuilder::new(client, StartOffset::At(start_offset)) .with_max_batch_size(self.config.max_batch_size.as_bytes() as i32) - .with_max_wait_ms(self.config.produce_record_timeout.as_millis() as i32) + .with_max_wait_ms(self.config.consumer_wait_timeout.as_millis() as i32) .build(); debug!( From d87ab06b280486b5370e0ba83fa4be8aa23b6457 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Tue, 2 Jan 2024 16:38:43 +0900 Subject: [PATCH 17/21] feat: add kafka wal integration test utils (#3069) * feat(tests-integration): add wal_config * feat: add kafka wal integration test utils --- tests-integration/src/cluster.rs | 22 ++++- tests-integration/src/lib.rs | 2 +- tests-integration/src/standalone.rs | 20 +++- tests-integration/src/test_util.rs | 10 +- tests-integration/src/tests.rs | 2 + tests-integration/src/tests/test_util.rs | 115 ++++++++++++++++++++++- 6 files changed, 161 insertions(+), 10 deletions(-) diff --git a/tests-integration/src/cluster.rs b/tests-integration/src/cluster.rs index cf0d1fd02e27..ae54a00d986a 100644 --- a/tests-integration/src/cluster.rs +++ b/tests-integration/src/cluster.rs @@ -22,6 +22,7 @@ use catalog::kvbackend::{CachedMetaKvBackend, MetaKvBackend}; use client::client_manager::DatanodeClients; use client::Client; use common_base::Plugins; +use common_config::WalConfig; use common_grpc::channel_manager::{ChannelConfig, ChannelManager}; use common_meta::heartbeat::handler::parse_mailbox_message::ParseMailboxMessageHandler; use common_meta::heartbeat::handler::HandlerGroupExecutor; @@ -64,12 +65,14 @@ pub struct GreptimeDbCluster { pub frontend: Arc, } +#[derive(Clone)] pub struct GreptimeDbClusterBuilder { cluster_name: String, kv_backend: KvBackendRef, store_config: Option, store_providers: Option>, datanodes: Option, + wal_config: WalConfig, } impl GreptimeDbClusterBuilder { @@ -95,6 +98,7 @@ impl GreptimeDbClusterBuilder { store_config: None, store_providers: None, datanodes: None, + wal_config: WalConfig::default(), } } @@ -113,6 +117,11 @@ impl GreptimeDbClusterBuilder { self } + pub fn with_wal_config(mut self, wal_config: WalConfig) -> Self { + self.wal_config = wal_config; + self + } + pub async fn build(self) -> GreptimeDbCluster { let datanodes = self.datanodes.unwrap_or(4); @@ -176,19 +185,27 @@ impl GreptimeDbClusterBuilder { for i in 0..datanodes { let datanode_id = i as u64 + 1; - + let mode = Mode::Distributed; let mut opts = if let Some(store_config) = &self.store_config { let home_tmp_dir = create_temp_dir(&format!("gt_home_{}", &self.cluster_name)); let home_dir = home_tmp_dir.path().to_str().unwrap().to_string(); dir_guards.push(FileDirGuard::new(home_tmp_dir)); - create_datanode_opts(store_config.clone(), vec![], home_dir) + create_datanode_opts( + mode, + store_config.clone(), + vec![], + home_dir, + self.wal_config.clone(), + ) } else { let (opts, guard) = create_tmp_dir_and_datanode_opts( + mode, StorageType::File, self.store_providers.clone().unwrap_or_default(), &format!("{}-dn-{}", self.cluster_name, datanode_id), + self.wal_config.clone(), ); storage_guards.push(guard.storage_guards); @@ -197,7 +214,6 @@ impl GreptimeDbClusterBuilder { opts }; opts.node_id = Some(datanode_id); - opts.mode = Mode::Distributed; let datanode = self.create_datanode(opts, meta_srv.clone()).await; diff --git a/tests-integration/src/lib.rs b/tests-integration/src/lib.rs index b0b28ba4651c..730694b8c67f 100644 --- a/tests-integration/src/lib.rs +++ b/tests-integration/src/lib.rs @@ -21,7 +21,7 @@ mod otlp; mod prom_store; pub mod test_util; -mod standalone; +pub mod standalone; #[cfg(test)] mod tests; diff --git a/tests-integration/src/standalone.rs b/tests-integration/src/standalone.rs index 072ff2282099..20348c462aa0 100644 --- a/tests-integration/src/standalone.rs +++ b/tests-integration/src/standalone.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use cmd::options::MixOptions; use common_base::Plugins; use common_catalog::consts::MIN_USER_TABLE_ID; -use common_config::KvBackendConfig; +use common_config::{KvBackendConfig, WalConfig}; use common_meta::cache_invalidator::DummyCacheInvalidator; use common_meta::ddl::table_meta::TableMetadataAllocator; use common_meta::ddl_manager::DdlManager; @@ -32,6 +32,7 @@ use datanode::datanode::DatanodeBuilder; use frontend::frontend::FrontendOptions; use frontend::instance::builder::FrontendBuilder; use frontend::instance::{FrontendInstance, Instance, StandaloneDatanodeManager}; +use servers::Mode; use crate::test_util::{self, create_tmp_dir_and_datanode_opts, StorageType, TestGuard}; @@ -42,8 +43,10 @@ pub struct GreptimeDbStandalone { pub guard: TestGuard, } +#[derive(Clone)] pub struct GreptimeDbStandaloneBuilder { instance_name: String, + wal_config: WalConfig, store_providers: Option>, default_store: Option, plugin: Option, @@ -56,6 +59,7 @@ impl GreptimeDbStandaloneBuilder { store_providers: None, plugin: None, default_store: None, + wal_config: WalConfig::default(), } } @@ -82,12 +86,22 @@ impl GreptimeDbStandaloneBuilder { } } + pub fn with_wal_config(mut self, wal_config: WalConfig) -> Self { + self.wal_config = wal_config; + self + } + pub async fn build(self) -> GreptimeDbStandalone { let default_store_type = self.default_store.unwrap_or(StorageType::File); let store_types = self.store_providers.unwrap_or_default(); - let (opts, guard) = - create_tmp_dir_and_datanode_opts(default_store_type, store_types, &self.instance_name); + let (opts, guard) = create_tmp_dir_and_datanode_opts( + Mode::Standalone, + default_store_type, + store_types, + &self.instance_name, + self.wal_config.clone(), + ); let procedure_config = ProcedureConfig::default(); let kv_backend_config = KvBackendConfig::default(); diff --git a/tests-integration/src/test_util.rs b/tests-integration/src/test_util.rs index 6bb91b89eb98..04e31d91ca3f 100644 --- a/tests-integration/src/test_util.rs +++ b/tests-integration/src/test_util.rs @@ -21,6 +21,7 @@ use std::time::Duration; use auth::UserProviderRef; use axum::Router; use catalog::kvbackend::KvBackendCatalogManager; +use common_config::WalConfig; use common_meta::key::catalog_name::CatalogNameKey; use common_meta::key::schema_name::SchemaNameKey; use common_query::Output; @@ -294,9 +295,11 @@ impl TestGuard { } pub fn create_tmp_dir_and_datanode_opts( + mode: Mode, default_store_type: StorageType, store_provider_types: Vec, name: &str, + wal_config: WalConfig, ) -> (DatanodeOptions, TestGuard) { let home_tmp_dir = create_temp_dir(&format!("gt_data_{name}")); let home_dir = home_tmp_dir.path().to_str().unwrap().to_string(); @@ -314,7 +317,7 @@ pub fn create_tmp_dir_and_datanode_opts( store_providers.push(store); storage_guards.push(StorageGuard(data_tmp_dir)) } - let opts = create_datanode_opts(default_store, store_providers, home_dir); + let opts = create_datanode_opts(mode, default_store, store_providers, home_dir, wal_config); ( opts, @@ -326,9 +329,11 @@ pub fn create_tmp_dir_and_datanode_opts( } pub(crate) fn create_datanode_opts( + mode: Mode, default_store: ObjectStoreConfig, providers: Vec, home_dir: String, + wal_config: WalConfig, ) -> DatanodeOptions { DatanodeOptions { node_id: Some(0), @@ -339,7 +344,8 @@ pub(crate) fn create_datanode_opts( store: default_store, ..Default::default() }, - mode: Mode::Standalone, + mode, + wal: wal_config, ..Default::default() } } diff --git a/tests-integration/src/tests.rs b/tests-integration/src/tests.rs index 692a9de8d6ab..8d1e421738aa 100644 --- a/tests-integration/src/tests.rs +++ b/tests-integration/src/tests.rs @@ -14,6 +14,8 @@ mod instance_test; mod promql_test; +// TODO(weny): Remove it. +#[allow(dead_code, unused_macros)] mod test_util; use std::collections::HashMap; diff --git a/tests-integration/src/tests/test_util.rs b/tests-integration/src/tests/test_util.rs index edf21ba7601d..32be423e69bb 100644 --- a/tests-integration/src/tests/test_util.rs +++ b/tests-integration/src/tests/test_util.rs @@ -12,10 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::env; use std::sync::Arc; +use common_config::wal::KafkaConfig; +use common_config::WalConfig; use common_query::Output; use common_recordbatch::util; +use common_telemetry::warn; use common_test_util::find_workspace_path; use frontend::instance::Instance; use rstest_reuse::{self, template}; @@ -25,7 +29,13 @@ use crate::standalone::{GreptimeDbStandalone, GreptimeDbStandaloneBuilder}; use crate::test_util::StorageType; use crate::tests::{create_distributed_instance, MockDistributedInstance}; -pub(crate) trait MockInstance { +#[async_trait::async_trait] +pub(crate) trait RebuildableMockInstance: MockInstance { + // Rebuilds the instance and returns rebuilt frontend instance. + async fn rebuild(&mut self) -> Arc; +} + +pub(crate) trait MockInstance: Sync + Send { fn frontend(&self) -> Arc; fn is_distributed_mode(&self) -> bool; @@ -51,6 +61,54 @@ impl MockInstance for MockDistributedInstance { } } +pub(crate) enum MockInstanceBuilder { + Standalone(GreptimeDbStandaloneBuilder), + Distributed(GreptimeDbClusterBuilder), +} + +impl MockInstanceBuilder { + async fn build(&self) -> Arc { + match self { + MockInstanceBuilder::Standalone(builder) => Arc::new(builder.clone().build().await), + MockInstanceBuilder::Distributed(builder) => { + Arc::new(MockDistributedInstance(builder.clone().build().await)) + } + } + } +} + +pub(crate) struct TestContext { + instance: Arc, + builder: MockInstanceBuilder, +} + +impl TestContext { + async fn new(builder: MockInstanceBuilder) -> Self { + let instance = builder.build().await; + + Self { instance, builder } + } +} + +#[async_trait::async_trait] +impl RebuildableMockInstance for TestContext { + async fn rebuild(&mut self) -> Arc { + let instance = self.builder.build().await; + self.instance = instance; + self.instance.frontend() + } +} + +impl MockInstance for TestContext { + fn frontend(&self) -> Arc { + self.instance.frontend() + } + + fn is_distributed_mode(&self) -> bool { + self.instance.is_distributed_mode() + } +} + pub(crate) async fn standalone() -> Arc { let test_name = uuid::Uuid::new_v4().to_string(); let instance = GreptimeDbStandaloneBuilder::new(&test_name).build().await; @@ -86,6 +144,61 @@ pub(crate) async fn distributed_with_multiple_object_stores() -> Arc Option> { + let _ = dotenv::dotenv(); + let endpoints = env::var("GT_KAFKA_ENDPOINTS").unwrap_or_default(); + common_telemetry::init_default_ut_logging(); + if endpoints.is_empty() { + warn!("The endpoints is empty, skipping the test"); + return None; + } + + let endpoints = endpoints.split(',').map(|s| s.trim().to_string()).collect(); + let test_name = uuid::Uuid::new_v4().to_string(); + let builder = GreptimeDbStandaloneBuilder::new(&test_name).with_wal_config(WalConfig::Kafka( + KafkaConfig { + broker_endpoints: endpoints, + ..Default::default() + }, + )); + let instance = TestContext::new(MockInstanceBuilder::Standalone(builder)).await; + Some(Box::new(instance)) +} + +pub(crate) async fn distributed_with_kafka_wal() -> Option> { + let _ = dotenv::dotenv(); + let endpoints = env::var("GT_KAFKA_ENDPOINTS").unwrap_or_default(); + common_telemetry::init_default_ut_logging(); + if endpoints.is_empty() { + warn!("The endpoints is empty, skipping the test"); + return None; + } + + let endpoints = endpoints.split(',').map(|s| s.trim().to_string()).collect(); + let test_name = uuid::Uuid::new_v4().to_string(); + let builder = GreptimeDbClusterBuilder::new(&test_name) + .await + .with_wal_config(WalConfig::Kafka(KafkaConfig { + broker_endpoints: endpoints, + ..Default::default() + })); + let instance = TestContext::new(MockInstanceBuilder::Distributed(builder)).await; + Some(Box::new(instance)) +} + +#[template] +#[rstest] +#[case::test_with_standalone(standalone_with_kafka_wal())] +#[case::test_with_distributed(distributed_with_kafka_wal())] +#[awt] +#[tokio::test(flavor = "multi_thread")] +pub(crate) fn both_instances_cases_with_kafka_wal( + #[future] + #[case] + instance: Arc, +) { +} + #[template] #[rstest] #[case::test_with_standalone(standalone_with_multiple_object_stores())] From 2b181e91e03d69017ca523e758019fee6d15fd11 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Tue, 2 Jan 2024 16:40:02 +0900 Subject: [PATCH 18/21] refactor: unify the injection of WAL option (#3066) * feat: add prepare_wal_option * refactor: use integer hashmap * feat: unify the injection of WAL option * fix: fix procedure_flow_upgrade_candidate_with_retry * chore: apply suggestions from CR --- src/common/meta/src/ddl/create_table.rs | 10 +-- src/common/meta/src/instruction.rs | 6 +- src/common/meta/src/key.rs | 2 +- src/common/meta/src/key/datanode_table.rs | 75 ++++++++++++++++--- src/common/meta/src/wal.rs | 13 ++++ src/datanode/src/datanode.rs | 15 ++-- src/meta-srv/src/error.rs | 8 ++ .../region_failover/activate_region.rs | 33 ++++---- .../region_failover/update_metadata.rs | 5 +- .../src/procedure/region_migration.rs | 75 ++++++++++++++----- .../region_migration/migration_start.rs | 12 +-- .../region_migration/open_candidate_region.rs | 31 ++++---- .../upgrade_candidate_region.rs | 22 ++---- 13 files changed, 204 insertions(+), 103 deletions(-) diff --git a/src/common/meta/src/ddl/create_table.rs b/src/common/meta/src/ddl/create_table.rs index c6e09006b470..b480c82acdfd 100644 --- a/src/common/meta/src/ddl/create_table.rs +++ b/src/common/meta/src/ddl/create_table.rs @@ -20,7 +20,6 @@ use api::v1::region::{ }; use api::v1::{ColumnDef, SemanticType}; use async_trait::async_trait; -use common_config::WAL_OPTIONS_KEY; use common_error::ext::BoxedError; use common_procedure::error::{ ExternalSnafu, FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu, @@ -48,6 +47,7 @@ use crate::rpc::ddl::CreateTableTask; use crate::rpc::router::{ find_leader_regions, find_leaders, operating_leader_regions, RegionRoute, }; +use crate::wal::prepare_wal_option; pub struct CreateTableProcedure { pub context: DdlContext, @@ -455,13 +455,7 @@ impl CreateRequestBuilder { request.region_id = region_id.as_u64(); request.path = storage_path; // Stores the encoded wal options into the request options. - region_wal_options - .get(®ion_id.region_number()) - .and_then(|wal_options| { - request - .options - .insert(WAL_OPTIONS_KEY.to_string(), wal_options.clone()) - }); + prepare_wal_option(&mut request.options, region_id, region_wal_options); if let Some(physical_table_id) = self.physical_table_id { // Logical table has the same region numbers with physical table, and they have a one-to-one mapping. diff --git a/src/common/meta/src/instruction.rs b/src/common/meta/src/instruction.rs index 8c0ba9ecfa7a..d0f5c9a27d1a 100644 --- a/src/common/meta/src/instruction.rs +++ b/src/common/meta/src/instruction.rs @@ -92,13 +92,15 @@ impl Display for OpenRegion { } } +#[serde_with::serde_as] #[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] pub struct OpenRegion { pub region_ident: RegionIdent, pub region_storage_path: String, pub region_options: HashMap, #[serde(default)] - pub region_wal_options: HashMap, + #[serde_as(as = "HashMap")] + pub region_wal_options: HashMap, #[serde(default)] pub skip_wal_replay: bool, } @@ -108,7 +110,7 @@ impl OpenRegion { region_ident: RegionIdent, path: &str, region_options: HashMap, - region_wal_options: HashMap, + region_wal_options: HashMap, skip_wal_replay: bool, ) -> Self { Self { diff --git a/src/common/meta/src/key.rs b/src/common/meta/src/key.rs index 57de421be202..abe116f973e5 100644 --- a/src/common/meta/src/key.rs +++ b/src/common/meta/src/key.rs @@ -604,7 +604,7 @@ impl TableMetadataManager { current_table_route_value: &DeserializedValueWithBytes, new_region_routes: Vec, new_region_options: &HashMap, - new_region_wal_options: &HashMap, + new_region_wal_options: &HashMap, ) -> Result<()> { // Updates the datanode table key value pairs. let current_region_distribution = diff --git a/src/common/meta/src/key/datanode_table.rs b/src/common/meta/src/key/datanode_table.rs index b2e25e014bc8..3000d56a5135 100644 --- a/src/common/meta/src/key/datanode_table.rs +++ b/src/common/meta/src/key/datanode_table.rs @@ -34,6 +34,7 @@ use crate::rpc::store::RangeRequest; use crate::rpc::KeyValue; use crate::DatanodeId; +#[serde_with::serde_as] #[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Default)] /// RegionInfo /// For compatible reason, DON'T modify the field name. @@ -48,14 +49,15 @@ pub struct RegionInfo { #[serde(default)] pub region_options: HashMap, /// The per-region wal options. - /// Key: region number (in string representation). Value: the encoded wal options of the region. + /// Key: region number. Value: the encoded wal options of the region. #[serde(default)] - pub region_wal_options: HashMap, + #[serde_as(as = "HashMap")] + pub region_wal_options: HashMap, } pub struct DatanodeTableKey { - datanode_id: DatanodeId, - table_id: TableId, + pub datanode_id: DatanodeId, + pub table_id: TableId, } impl DatanodeTableKey { @@ -181,7 +183,7 @@ impl DatanodeTableManager { .filter_map(|region_number| { region_wal_options .get(region_number) - .map(|wal_options| (region_number.to_string(), wal_options.clone())) + .map(|wal_options| (*region_number, wal_options.clone())) }) .collect(); @@ -214,7 +216,7 @@ impl DatanodeTableManager { current_region_distribution: RegionDistribution, new_region_distribution: RegionDistribution, new_region_options: &HashMap, - new_region_wal_options: &HashMap, + new_region_wal_options: &HashMap, ) -> Result { let mut opts = Vec::new(); @@ -306,6 +308,61 @@ mod tests { assert!(parsed.is_ok()); } + #[derive(Debug, Serialize, Deserialize, PartialEq)] + struct StringHashMap { + inner: HashMap, + } + + #[serde_with::serde_as] + #[derive(Debug, Serialize, Deserialize, PartialEq)] + struct IntegerHashMap { + #[serde_as(as = "HashMap")] + inner: HashMap, + } + + #[test] + fn test_serde_with_integer_hash_map() { + let map = StringHashMap { + inner: HashMap::from([ + ("1".to_string(), "aaa".to_string()), + ("2".to_string(), "bbb".to_string()), + ("3".to_string(), "ccc".to_string()), + ]), + }; + let encoded = serde_json::to_string(&map).unwrap(); + let decoded: IntegerHashMap = serde_json::from_str(&encoded).unwrap(); + assert_eq!( + IntegerHashMap { + inner: HashMap::from([ + (1, "aaa".to_string()), + (2, "bbb".to_string()), + (3, "ccc".to_string()), + ]), + }, + decoded + ); + + let map = IntegerHashMap { + inner: HashMap::from([ + (1, "aaa".to_string()), + (2, "bbb".to_string()), + (3, "ccc".to_string()), + ]), + }; + let encoded = serde_json::to_string(&map).unwrap(); + let decoded: StringHashMap = serde_json::from_str(&encoded).unwrap(); + assert_eq!( + StringHashMap { + inner: HashMap::from([ + ("1".to_string(), "aaa".to_string()), + ("2".to_string(), "bbb".to_string()), + ("3".to_string(), "ccc".to_string()), + ]), + }, + decoded + ); + } + // This test intends to ensure both the `serde_json::to_string` + `serde_json::from_str` // and `serde_json::to_vec` + `serde_json::from_slice` work for `DatanodeTableValue`. // Warning: if the key of `region_wal_options` is of type non-String, this test would fail. @@ -320,9 +377,9 @@ mod tests { ("c".to_string(), "cc".to_string()), ]), region_wal_options: HashMap::from([ - ("1".to_string(), "aaa".to_string()), - ("2".to_string(), "bbb".to_string()), - ("3".to_string(), "ccc".to_string()), + (1, "aaa".to_string()), + (2, "bbb".to_string()), + (3, "ccc".to_string()), ]), }; let table_value = DatanodeTableValue { diff --git a/src/common/meta/src/wal.rs b/src/common/meta/src/wal.rs index 853c6fa5df63..c7af1d64d306 100644 --- a/src/common/meta/src/wal.rs +++ b/src/common/meta/src/wal.rs @@ -18,8 +18,11 @@ pub mod options_allocator; use std::collections::HashMap; use common_config::wal::StandaloneWalConfig; +use common_config::WAL_OPTIONS_KEY; +use common_telemetry::warn; use serde::{Deserialize, Serialize}; use serde_with::with_prefix; +use store_api::storage::{RegionId, RegionNumber}; use crate::error::Result; use crate::wal::kafka::KafkaConfig; @@ -55,6 +58,16 @@ impl From for WalConfig { } } +pub fn prepare_wal_option( + options: &mut HashMap, + region_id: RegionId, + region_wal_options: &HashMap, +) { + if let Some(wal_options) = region_wal_options.get(®ion_id.region_number()) { + options.insert(WAL_OPTIONS_KEY.to_string(), wal_options.clone()); + } +} + #[cfg(test)] mod tests { use std::time::Duration; diff --git a/src/datanode/src/datanode.rs b/src/datanode/src/datanode.rs index 0a0206eddc66..12c28fb16173 100644 --- a/src/datanode/src/datanode.rs +++ b/src/datanode/src/datanode.rs @@ -22,11 +22,12 @@ use std::sync::Arc; use catalog::memory::MemoryCatalogManager; use common_base::Plugins; use common_config::wal::{KafkaConfig, RaftEngineConfig}; -use common_config::{WalConfig, WAL_OPTIONS_KEY}; +use common_config::WalConfig; use common_error::ext::BoxedError; use common_greptimedb_telemetry::GreptimeDBTelemetryTask; use common_meta::key::datanode_table::{DatanodeTableManager, DatanodeTableValue}; use common_meta::kv_backend::KvBackendRef; +use common_meta::wal::prepare_wal_option; pub use common_procedure::options::ProcedureConfig; use common_runtime::Runtime; use common_telemetry::{error, info, warn}; @@ -538,13 +539,11 @@ async fn open_all_regions( for region_number in table_value.regions { // Augments region options with wal options if a wal options is provided. let mut region_options = table_value.region_info.region_options.clone(); - table_value - .region_info - .region_wal_options - .get(®ion_number.to_string()) - .and_then(|wal_options| { - region_options.insert(WAL_OPTIONS_KEY.to_string(), wal_options.clone()) - }); + prepare_wal_option( + &mut region_options, + RegionId::new(table_value.table_id, region_number), + &table_value.region_info.region_wal_options, + ); regions.push(( RegionId::new(table_value.table_id, region_number), diff --git a/src/meta-srv/src/error.rs b/src/meta-srv/src/error.rs index 5272c3abe77a..f0c29a46df7f 100644 --- a/src/meta-srv/src/error.rs +++ b/src/meta-srv/src/error.rs @@ -327,6 +327,13 @@ pub enum Error { location: Location, }, + #[snafu(display("Datanode table not found: {}, datanode: {}", table_id, datanode_id))] + DatanodeTableNotFound { + table_id: TableId, + datanode_id: DatanodeId, + location: Location, + }, + #[snafu(display("Table route corrupted, key: {}, reason: {}", key, reason))] CorruptedTableRoute { key: String, @@ -683,6 +690,7 @@ impl ErrorExt for Error { | Error::InvalidRegionKeyFromUtf8 { .. } | Error::TableRouteNotFound { .. } | Error::TableInfoNotFound { .. } + | Error::DatanodeTableNotFound { .. } | Error::CorruptedTableRoute { .. } | Error::MoveValue { .. } | Error::InvalidUtf8Value { .. } diff --git a/src/meta-srv/src/procedure/region_failover/activate_region.rs b/src/meta-srv/src/procedure/region_failover/activate_region.rs index a107ebfb0315..a2b1c8fd9303 100644 --- a/src/meta-srv/src/procedure/region_failover/activate_region.rs +++ b/src/meta-srv/src/procedure/region_failover/activate_region.rs @@ -17,13 +17,14 @@ use std::time::Duration; use api::v1::meta::MailboxMessage; use async_trait::async_trait; -use common_meta::ddl::utils::region_storage_path; use common_meta::instruction::{Instruction, InstructionReply, OpenRegion, SimpleReply}; +use common_meta::key::datanode_table::{DatanodeTableKey, RegionInfo}; use common_meta::peer::Peer; use common_meta::RegionIdent; use common_telemetry::{debug, info}; use serde::{Deserialize, Serialize}; use snafu::{OptionExt, ResultExt}; +use store_api::storage::RegionNumber; use super::update_metadata::UpdateRegionMetadata; use super::{RegionFailoverContext, State}; @@ -44,7 +45,7 @@ pub(super) struct ActivateRegion { // An `None` option stands for uninitialized. region_storage_path: Option, region_options: Option>, - region_wal_options: Option>, + region_wal_options: Option>, } impl ActivateRegion { @@ -65,27 +66,31 @@ impl ActivateRegion { timeout: Duration, ) -> Result { let table_id = failed_region.table_id; - let table_info = ctx + // Retrieves the wal options from failed datanode table value. + let datanode_table_value = ctx .table_metadata_manager - .table_info_manager() - .get(table_id) + .datanode_table_manager() + .get(&DatanodeTableKey::new(failed_region.datanode_id, table_id)) .await .context(error::TableMetadataManagerSnafu)? - .context(error::TableInfoNotFoundSnafu { table_id })? - .into_inner() - .table_info; - - let region_storage_path = - region_storage_path(&table_info.catalog_name, &table_info.schema_name); + .context(error::DatanodeTableNotFoundSnafu { + table_id, + datanode_id: failed_region.datanode_id, + })?; let candidate_ident = RegionIdent { datanode_id: self.candidate.id, ..failed_region.clone() }; info!("Activating region: {candidate_ident:?}"); - let region_options: HashMap = (&table_info.meta.options).into(); - // TODO(niebayes): properly fetch or construct region wal options. - let region_wal_options = HashMap::new(); + + let RegionInfo { + region_storage_path, + region_options, + region_wal_options, + .. + } = datanode_table_value.region_info; + let instruction = Instruction::OpenRegion(OpenRegion::new( candidate_ident.clone(), ®ion_storage_path, diff --git a/src/meta-srv/src/procedure/region_failover/update_metadata.rs b/src/meta-srv/src/procedure/region_failover/update_metadata.rs index c2218c6afede..254ba3e8107d 100644 --- a/src/meta-srv/src/procedure/region_failover/update_metadata.rs +++ b/src/meta-srv/src/procedure/region_failover/update_metadata.rs @@ -23,6 +23,7 @@ use common_meta::RegionIdent; use common_telemetry::info; use serde::{Deserialize, Serialize}; use snafu::{OptionExt, ResultExt}; +use store_api::storage::RegionNumber; use super::invalidate_cache::InvalidateCache; use super::{RegionFailoverContext, State}; @@ -36,7 +37,7 @@ pub(super) struct UpdateRegionMetadata { region_storage_path: String, region_options: HashMap, #[serde(default)] - region_wal_options: HashMap, + region_wal_options: HashMap, } impl UpdateRegionMetadata { @@ -44,7 +45,7 @@ impl UpdateRegionMetadata { candidate: Peer, region_storage_path: String, region_options: HashMap, - region_wal_options: HashMap, + region_wal_options: HashMap, ) -> Self { Self { candidate, diff --git a/src/meta-srv/src/procedure/region_migration.rs b/src/meta-srv/src/procedure/region_migration.rs index b187a026723a..fbbf19822555 100644 --- a/src/meta-srv/src/procedure/region_migration.rs +++ b/src/meta-srv/src/procedure/region_migration.rs @@ -31,6 +31,7 @@ use std::time::Duration; use api::v1::meta::MailboxMessage; use common_meta::instruction::Instruction; +use common_meta::key::datanode_table::{DatanodeTableKey, DatanodeTableValue}; use common_meta::key::table_info::TableInfoValue; use common_meta::key::table_route::TableRouteValue; use common_meta::key::{DeserializedValueWithBytes, TableMetadataManagerRef}; @@ -90,6 +91,8 @@ pub struct VolatileContext { opening_region_guard: Option, /// `table_route` is stored via previous steps for future use. table_route: Option>, + /// `datanode_table` is stored via previous steps for future use. + from_peer_datanode_table: Option, /// `table_info` is stored via previous steps for future use. /// /// `table_info` should remain unchanged during the procedure; @@ -250,6 +253,42 @@ impl Context { Ok(table_info_value.as_ref().unwrap()) } + /// Returns the `table_info` of [VolatileContext] if any. + /// Otherwise, returns the value retrieved from remote. + /// + /// Retry: + /// - Failed to retrieve the metadata of datanode. + pub async fn get_from_peer_datanode_table_value(&mut self) -> Result<&DatanodeTableValue> { + let datanode_value = &mut self.volatile_ctx.from_peer_datanode_table; + + if datanode_value.is_none() { + let table_id = self.persistent_ctx.region_id.table_id(); + let datanode_id = self.persistent_ctx.from_peer.id; + + let datanode_table = self + .table_metadata_manager + .datanode_table_manager() + .get(&DatanodeTableKey { + datanode_id, + table_id, + }) + .await + .context(error::TableMetadataManagerSnafu) + .map_err(|e| error::Error::RetryLater { + reason: e.to_string(), + location: location!(), + })? + .context(error::DatanodeTableNotFoundSnafu { + table_id, + datanode_id, + })?; + + *datanode_value = Some(datanode_table); + } + + Ok(datanode_value.as_ref().unwrap()) + } + /// Removes the `table_info` of [VolatileContext], returns true if any. pub fn remove_table_info_value(&mut self) -> bool { let value = self.volatile_ctx.table_info.take(); @@ -889,7 +928,7 @@ mod tests { to_peer_id, Arc::new(|id| Ok(new_open_region_reply(id, false, None))), )), - Assertion::error(|error| assert!(error.is_retryable())), + Assertion::error(|error| assert!(error.is_retryable(), "err: {error:?}")), ), // OpenCandidateRegion Step::next( @@ -942,25 +981,25 @@ mod tests { None, Assertion::simple(assert_region_migration_end, assert_done), ), + // RegionMigrationStart + Step::setup( + "Sets state to RegionMigrationStart", + merge_before_test_fn(vec![ + setup_state(Arc::new(|| Box::new(RegionMigrationStart))), + Arc::new(reset_volatile_ctx), + ]), + ), + // RegionMigrationEnd + // Note: We can't run this test multiple times; + // the `peer_id`'s `DatanodeTable` will be removed after first-time migration success. + Step::next( + "Should be the region migration end(has been migrated)", + None, + Assertion::simple(assert_region_migration_end, assert_done), + ), ]; - let setup_to_latest_persisted_state = Step::setup( - "Sets state to OpenCandidateRegion", - merge_before_test_fn(vec![ - setup_state(Arc::new(|| Box::new(OpenCandidateRegion))), - Arc::new(reset_volatile_ctx), - ]), - ); - - let steps = [ - steps.clone(), - vec![setup_to_latest_persisted_state.clone()], - steps.clone()[1..].to_vec(), - vec![setup_to_latest_persisted_state], - steps.clone()[1..].to_vec(), - ] - .concat(); - + let steps = [steps.clone()].concat(); let timer = Instant::now(); // Run the table tests. diff --git a/src/meta-srv/src/procedure/region_migration/migration_start.rs b/src/meta-srv/src/procedure/region_migration/migration_start.rs index 68b291cb87c1..6bacc92433aa 100644 --- a/src/meta-srv/src/procedure/region_migration/migration_start.rs +++ b/src/meta-srv/src/procedure/region_migration/migration_start.rs @@ -52,7 +52,7 @@ impl State for RegionMigrationStart { let region_route = self.retrieve_region_route(ctx, region_id).await?; let to_peer = &ctx.persistent_ctx.to_peer; - if self.check_leader_region_on_peer(®ion_route, to_peer)? { + if self.has_migrated(®ion_route, to_peer)? { Ok((Box::new(RegionMigrationEnd), Status::Done)) } else if self.check_candidate_region_on_peer(®ion_route, to_peer) { Ok((Box::new(UpdateMetadata::Downgrade), Status::executing(true))) @@ -112,16 +112,12 @@ impl RegionMigrationStart { region_opened } - /// Checks whether the leader region on region has been opened. - /// Returns true if it's been opened. + /// Checks whether the region has been migrated. + /// Returns true if it's. /// /// Abort(non-retry): /// - Leader peer of RegionRoute is not found. - fn check_leader_region_on_peer( - &self, - region_route: &RegionRoute, - to_peer: &Peer, - ) -> Result { + fn has_migrated(&self, region_route: &RegionRoute, to_peer: &Peer) -> Result { let region_id = region_route.region.id; let region_opened = region_route diff --git a/src/meta-srv/src/procedure/region_migration/open_candidate_region.rs b/src/meta-srv/src/procedure/region_migration/open_candidate_region.rs index 74b904ce0105..176db54952b3 100644 --- a/src/meta-srv/src/procedure/region_migration/open_candidate_region.rs +++ b/src/meta-srv/src/procedure/region_migration/open_candidate_region.rs @@ -13,13 +13,12 @@ // limitations under the License. use std::any::Any; -use std::collections::HashMap; use std::time::Duration; use api::v1::meta::MailboxMessage; -use common_meta::ddl::utils::region_storage_path; use common_meta::distributed_time_constants::MAILBOX_RTT_SECS; use common_meta::instruction::{Instruction, InstructionReply, OpenRegion, SimpleReply}; +use common_meta::key::datanode_table::RegionInfo; use common_meta::RegionIdent; use common_procedure::Status; use serde::{Deserialize, Serialize}; @@ -58,26 +57,21 @@ impl OpenCandidateRegion { /// Builds open region instructions /// /// Abort(non-retry): - /// - Table Info is not found. + /// - Datanode Table is not found. async fn build_open_region_instruction(&self, ctx: &mut Context) -> Result { let pc = &ctx.persistent_ctx; let cluster_id = pc.cluster_id; let table_id = pc.region_id.table_id(); let region_number = pc.region_id.region_number(); let candidate_id = pc.to_peer.id; + let datanode_table_value = ctx.get_from_peer_datanode_table_value().await?; - let table_info_value = ctx.get_table_info_value().await?; - let table_info = &table_info_value.table_info; - - // The region storage path is immutable after the region is created. - // Therefore, it's safe to store it in `VolatileContext` for future use. - let region_storage_path = - region_storage_path(&table_info.catalog_name, &table_info.schema_name); - - let engine = table_info.meta.engine.clone(); - let region_options: HashMap = (&table_info.meta.options).into(); - // TODO(niebayes): properly fetch or construct region wal options. - let region_wal_options = HashMap::new(); + let RegionInfo { + region_storage_path, + region_options, + region_wal_options, + engine, + } = datanode_table_value.region_info.clone(); let open_instruction = Instruction::OpenRegion(OpenRegion::new( RegionIdent { @@ -185,6 +179,7 @@ impl OpenCandidateRegion { #[cfg(test)] mod tests { use std::assert_matches::assert_matches; + use std::collections::HashMap; use common_catalog::consts::MITO2_ENGINE; use common_meta::key::table_route::TableRouteValue; @@ -222,7 +217,7 @@ mod tests { } #[tokio::test] - async fn test_table_info_is_not_found_error() { + async fn test_datanode_table_is_not_found_error() { let state = OpenCandidateRegion; let persistent_context = new_persistent_context(); let env = TestingEnv::new(); @@ -233,7 +228,7 @@ mod tests { .await .unwrap_err(); - assert_matches!(err, Error::TableInfoNotFound { .. }); + assert_matches!(err, Error::DatanodeTableNotFound { .. }); assert!(!err.is_retryable()); } @@ -405,7 +400,7 @@ mod tests { let table_info = new_test_table_info(1024, vec![1]).into(); let region_routes = vec![RegionRoute { region: Region::new_test(persistent_context.region_id), - leader_peer: Some(Peer::empty(3)), + leader_peer: Some(Peer::empty(1)), ..Default::default() }]; diff --git a/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs b/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs index 745b8487a8f3..793efa4db66e 100644 --- a/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs +++ b/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs @@ -12,9 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; - -use common_meta::ddl::utils::region_storage_path; use common_meta::key::datanode_table::RegionInfo; use common_meta::rpc::router::RegionRoute; use common_telemetry::{info, warn}; @@ -138,18 +135,13 @@ impl UpdateMetadata { } let region_routes = self.build_upgrade_candidate_region_metadata(ctx).await?; - let table_info_value = ctx.get_table_info_value().await?; - - let table_info = &table_info_value.table_info; - let region_storage_path = - region_storage_path(&table_info.catalog_name, &table_info.schema_name); - let engine = table_info.meta.engine.clone(); - let region_options: HashMap = (&table_info.meta.options).into(); - - // TODO(niebayes): properly fetch or construct region wal options. - let region_wal_options = HashMap::new(); - - // No remote fetch. + let datanode_table_value = ctx.get_from_peer_datanode_table_value().await?; + let RegionInfo { + region_storage_path, + region_options, + region_wal_options, + engine, + } = datanode_table_value.region_info.clone(); let table_route_value = ctx.get_table_route_value().await?; if let Err(err) = table_metadata_manager From 128d3717fab12c4af362de1e4854b94c5c021685 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Tue, 2 Jan 2024 18:05:20 +0900 Subject: [PATCH 19/21] test(tests-integration): add a naive test with kafka wal (#3071) * chore(tests-integration): add setup tests with kafka wal to README.md * feat(tests-integration): add meta wal config * fix(tests-integration): fix sign of both_instances_cases_with_kafka_wal * chore(tests-integration): set num_topic to 3 for tests * test(tests-integration): add a naive test with kafka wal * chore: apply suggestions from CR --- .env.example | 2 + tests-integration/README.md | 25 ++++- tests-integration/src/cluster.rs | 37 ++++--- tests-integration/src/standalone.rs | 11 ++- tests-integration/src/tests.rs | 3 +- .../src/tests/instance_kafka_wal_test.rs | 97 +++++++++++++++++++ tests-integration/src/tests/test_util.rs | 35 +++++-- 7 files changed, 184 insertions(+), 26 deletions(-) create mode 100644 tests-integration/src/tests/instance_kafka_wal_test.rs diff --git a/.env.example b/.env.example index 4d45913df043..2f51a7cc6559 100644 --- a/.env.example +++ b/.env.example @@ -19,3 +19,5 @@ GT_GCS_BUCKET = GCS bucket GT_GCS_SCOPE = GCS scope GT_GCS_CREDENTIAL_PATH = GCS credential path GT_GCS_ENDPOINT = GCS end point +# Settings for kafka wal test +GT_KAFKA_ENDPOINTS = localhost:9092 diff --git a/tests-integration/README.md b/tests-integration/README.md index 27f66da86780..a1622174a1b6 100644 --- a/tests-integration/README.md +++ b/tests-integration/README.md @@ -1,4 +1,4 @@ -## Setup +## Setup tests for multiple storage backend To run the integration test, please copy `.env.example` to `.env` in the project root folder and change the values on need. @@ -13,7 +13,7 @@ GT_S3_ACCESS_KEY=S3 secret access key ``` -## Run +### Run Execute the following command in the project root folder: @@ -37,4 +37,23 @@ Test azblob storage: ``` cargo test azblob -``` \ No newline at end of file +``` + +## Setup tests with Kafka wal + +To run the integration test, please copy `.env.example` to `.env` in the project root folder and change the values on need. + +```sh +GT_KAFKA_ENDPOINTS = localhost:9092 +``` + +### Setup kafka standalone + +``` +cd tests-integration/fixtures/kafka + +docker compose -f docker-compose-standalone.yml up +``` + + + diff --git a/tests-integration/src/cluster.rs b/tests-integration/src/cluster.rs index ae54a00d986a..336d1d68d0b0 100644 --- a/tests-integration/src/cluster.rs +++ b/tests-integration/src/cluster.rs @@ -31,6 +31,7 @@ use common_meta::kv_backend::etcd::EtcdStore; use common_meta::kv_backend::memory::MemoryKvBackend; use common_meta::kv_backend::KvBackendRef; use common_meta::peer::Peer; +use common_meta::wal::WalConfig as MetaWalConfig; use common_meta::DatanodeId; use common_runtime::Builder as RuntimeBuilder; use common_test_util::temp_dir::create_temp_dir; @@ -73,6 +74,7 @@ pub struct GreptimeDbClusterBuilder { store_providers: Option>, datanodes: Option, wal_config: WalConfig, + meta_wal_config: MetaWalConfig, } impl GreptimeDbClusterBuilder { @@ -99,6 +101,7 @@ impl GreptimeDbClusterBuilder { store_providers: None, datanodes: None, wal_config: WalConfig::default(), + meta_wal_config: MetaWalConfig::default(), } } @@ -122,13 +125,29 @@ impl GreptimeDbClusterBuilder { self } + pub fn with_meta_wal_config(mut self, wal_meta: MetaWalConfig) -> Self { + self.meta_wal_config = wal_meta; + self + } + pub async fn build(self) -> GreptimeDbCluster { let datanodes = self.datanodes.unwrap_or(4); let channel_config = ChannelConfig::new().timeout(Duration::from_secs(20)); let datanode_clients = Arc::new(DatanodeClients::new(channel_config)); - let meta_srv = self.build_metasrv(datanode_clients.clone()).await; + let opt = MetaSrvOptions { + procedure: ProcedureConfig { + // Due to large network delay during cross data-center. + // We only make max_retry_times and retry_delay large than the default in tests. + max_retry_times: 5, + retry_delay: Duration::from_secs(1), + }, + wal: self.meta_wal_config.clone(), + ..Default::default() + }; + + let meta_srv = self.build_metasrv(opt, datanode_clients.clone()).await; let (datanode_instances, storage_guards, dir_guards) = self.build_datanodes(meta_srv.clone(), datanodes).await; @@ -156,17 +175,11 @@ impl GreptimeDbClusterBuilder { } } - async fn build_metasrv(&self, datanode_clients: Arc) -> MockInfo { - let opt = MetaSrvOptions { - procedure: ProcedureConfig { - // Due to large network delay during cross data-center. - // We only make max_retry_times and retry_delay large than the default in tests. - max_retry_times: 5, - retry_delay: Duration::from_secs(1), - }, - ..Default::default() - }; - + async fn build_metasrv( + &self, + opt: MetaSrvOptions, + datanode_clients: Arc, + ) -> MockInfo { meta_srv::mocks::mock(opt, self.kv_backend.clone(), None, Some(datanode_clients)).await } diff --git a/tests-integration/src/standalone.rs b/tests-integration/src/standalone.rs index 20348c462aa0..9c6956a6bbda 100644 --- a/tests-integration/src/standalone.rs +++ b/tests-integration/src/standalone.rs @@ -24,7 +24,7 @@ use common_meta::ddl_manager::DdlManager; use common_meta::key::TableMetadataManager; use common_meta::region_keeper::MemoryRegionKeeper; use common_meta::sequence::SequenceBuilder; -use common_meta::wal::{WalConfig as MetaSrvWalConfig, WalOptionsAllocator}; +use common_meta::wal::{WalConfig as MetaWalConfig, WalOptionsAllocator}; use common_procedure::options::ProcedureConfig; use common_telemetry::logging::LoggingOptions; use datanode::config::DatanodeOptions; @@ -47,6 +47,7 @@ pub struct GreptimeDbStandalone { pub struct GreptimeDbStandaloneBuilder { instance_name: String, wal_config: WalConfig, + meta_wal_config: MetaWalConfig, store_providers: Option>, default_store: Option, plugin: Option, @@ -60,6 +61,7 @@ impl GreptimeDbStandaloneBuilder { plugin: None, default_store: None, wal_config: WalConfig::default(), + meta_wal_config: MetaWalConfig::default(), } } @@ -91,6 +93,11 @@ impl GreptimeDbStandaloneBuilder { self } + pub fn with_meta_wal_config(mut self, wal_meta: MetaWalConfig) -> Self { + self.meta_wal_config = wal_meta; + self + } + pub async fn build(self) -> GreptimeDbStandalone { let default_store_type = self.default_store.unwrap_or(StorageType::File); let store_types = self.store_providers.unwrap_or_default(); @@ -132,7 +139,7 @@ impl GreptimeDbStandaloneBuilder { .step(10) .build(), ); - let wal_meta = MetaSrvWalConfig::default(); + let wal_meta = self.meta_wal_config.clone(); let wal_options_allocator = Arc::new(WalOptionsAllocator::new( wal_meta.clone(), kv_backend.clone(), diff --git a/tests-integration/src/tests.rs b/tests-integration/src/tests.rs index 8d1e421738aa..22c0d591de9a 100644 --- a/tests-integration/src/tests.rs +++ b/tests-integration/src/tests.rs @@ -12,10 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod instance_kafka_wal_test; mod instance_test; mod promql_test; -// TODO(weny): Remove it. -#[allow(dead_code, unused_macros)] mod test_util; use std::collections::HashMap; diff --git a/tests-integration/src/tests/instance_kafka_wal_test.rs b/tests-integration/src/tests/instance_kafka_wal_test.rs new file mode 100644 index 000000000000..c882f17f95f0 --- /dev/null +++ b/tests-integration/src/tests/instance_kafka_wal_test.rs @@ -0,0 +1,97 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use common_query::Output; +use common_recordbatch::util; +use datatypes::vectors::{TimestampMillisecondVector, VectorRef}; +use frontend::error::Result; +use frontend::instance::Instance; +use rstest::rstest; +use rstest_reuse::apply; +use servers::query_handler::sql::SqlQueryHandler; +use session::context::{QueryContext, QueryContextRef}; + +use crate::tests::test_util::*; + +#[apply(both_instances_cases_with_kafka_wal)] +async fn test_create_database_and_insert_query(instance: Option>) { + let Some(instance) = instance else { return }; + + let instance = instance.frontend(); + + let output = execute_sql(&instance, "create database test").await; + assert!(matches!(output, Output::AffectedRows(1))); + + let output = execute_sql( + &instance, + r#"create table greptime.test.demo( + host STRING, + cpu DOUBLE, + memory DOUBLE, + ts timestamp, + TIME INDEX(ts) +)"#, + ) + .await; + assert!(matches!(output, Output::AffectedRows(0))); + + let output = execute_sql( + &instance, + r#"insert into test.demo(host, cpu, memory, ts) values + ('host1', 66.6, 1024, 1655276557000), + ('host2', 88.8, 333.3, 1655276558000) + "#, + ) + .await; + assert!(matches!(output, Output::AffectedRows(2))); + + let query_output = execute_sql(&instance, "select ts from test.demo order by ts limit 1").await; + match query_output { + Output::Stream(s) => { + let batches = util::collect(s).await.unwrap(); + assert_eq!(1, batches[0].num_columns()); + assert_eq!( + Arc::new(TimestampMillisecondVector::from_vec(vec![ + 1655276557000_i64 + ])) as VectorRef, + *batches[0].column(0) + ); + } + _ => unreachable!(), + } +} + +async fn execute_sql(instance: &Arc, sql: &str) -> Output { + execute_sql_with(instance, sql, QueryContext::arc()).await +} + +async fn try_execute_sql_with( + instance: &Arc, + sql: &str, + query_ctx: QueryContextRef, +) -> Result { + instance.do_query(sql, query_ctx).await.remove(0) +} + +async fn execute_sql_with( + instance: &Arc, + sql: &str, + query_ctx: QueryContextRef, +) -> Output { + try_execute_sql_with(instance, sql, query_ctx) + .await + .unwrap() +} diff --git a/tests-integration/src/tests/test_util.rs b/tests-integration/src/tests/test_util.rs index 32be423e69bb..5a2f41316be6 100644 --- a/tests-integration/src/tests/test_util.rs +++ b/tests-integration/src/tests/test_util.rs @@ -17,6 +17,8 @@ use std::sync::Arc; use common_config::wal::KafkaConfig; use common_config::WalConfig; +use common_meta::wal::kafka::KafkaConfig as MetaKafkaConfig; +use common_meta::wal::WalConfig as MetaWalConfig; use common_query::Output; use common_recordbatch::util; use common_telemetry::warn; @@ -61,6 +63,8 @@ impl MockInstance for MockDistributedInstance { } } +/// For test purpose. +#[allow(clippy::large_enum_variant)] pub(crate) enum MockInstanceBuilder { Standalone(GreptimeDbStandaloneBuilder), Distributed(GreptimeDbClusterBuilder), @@ -153,14 +157,22 @@ pub(crate) async fn standalone_with_kafka_wal() -> Option>(); let test_name = uuid::Uuid::new_v4().to_string(); - let builder = GreptimeDbStandaloneBuilder::new(&test_name).with_wal_config(WalConfig::Kafka( - KafkaConfig { + let builder = GreptimeDbStandaloneBuilder::new(&test_name) + .with_wal_config(WalConfig::Kafka(KafkaConfig { + broker_endpoints: endpoints.clone(), + ..Default::default() + })) + .with_meta_wal_config(MetaWalConfig::Kafka(MetaKafkaConfig { broker_endpoints: endpoints, + topic_name_prefix: test_name.to_string(), + num_topics: 3, ..Default::default() - }, - )); + })); let instance = TestContext::new(MockInstanceBuilder::Standalone(builder)).await; Some(Box::new(instance)) } @@ -174,12 +186,21 @@ pub(crate) async fn distributed_with_kafka_wal() -> Option>(); let test_name = uuid::Uuid::new_v4().to_string(); let builder = GreptimeDbClusterBuilder::new(&test_name) .await .with_wal_config(WalConfig::Kafka(KafkaConfig { + broker_endpoints: endpoints.clone(), + ..Default::default() + })) + .with_meta_wal_config(MetaWalConfig::Kafka(MetaKafkaConfig { broker_endpoints: endpoints, + topic_name_prefix: test_name.to_string(), + num_topics: 3, ..Default::default() })); let instance = TestContext::new(MockInstanceBuilder::Distributed(builder)).await; @@ -195,7 +216,7 @@ pub(crate) async fn distributed_with_kafka_wal() -> Option, + instance: Option>, ) { } From 2e686fe05342afed494a5addabce63a04e1110c2 Mon Sep 17 00:00:00 2001 From: Yingwen Date: Wed, 3 Jan 2024 10:05:06 +0800 Subject: [PATCH 20/21] feat(mito): Implement file cache (#3022) * feat: recover cache * feat: moka features * test: tests for file cache * chore: suppress warninig * fix: parse_inde_key consider suffix * feat: update cache * feat: expose cache file path * feat: use cache_path in test --- src/mito2/Cargo.toml | 2 +- src/mito2/src/cache.rs | 5 + src/mito2/src/cache/file_cache.rs | 407 ++++++++++++++++++++++++++++++ 3 files changed, 413 insertions(+), 1 deletion(-) create mode 100644 src/mito2/src/cache/file_cache.rs diff --git a/src/mito2/Cargo.toml b/src/mito2/Cargo.toml index a28e4f0426ea..e4868b047552 100644 --- a/src/mito2/Cargo.toml +++ b/src/mito2/Cargo.toml @@ -43,7 +43,7 @@ index.workspace = true lazy_static = "1.4" log-store = { workspace = true, optional = true } memcomparable = "0.2" -moka = { workspace = true, features = ["sync"] } +moka = { workspace = true, features = ["sync", "future"] } num_cpus = "1.13" object-store.workspace = true parquet = { workspace = true, features = ["async"] } diff --git a/src/mito2/src/cache.rs b/src/mito2/src/cache.rs index 529e5d3d4eee..cc02a2d037ce 100644 --- a/src/mito2/src/cache.rs +++ b/src/mito2/src/cache.rs @@ -15,6 +15,9 @@ //! Cache for the engine. mod cache_size; +// TODO(yingwen): Remove this after the write cache is ready. +#[allow(unused)] +pub(crate) mod file_cache; #[cfg(test)] pub(crate) mod test_util; @@ -38,6 +41,8 @@ const SST_META_TYPE: &str = "sst_meta"; const VECTOR_TYPE: &str = "vector"; // Metrics type key for pages. const PAGE_TYPE: &str = "page"; +// Metrics type key for files on the local store. +const FILE_TYPE: &str = "file"; /// Manages cached data for the engine. pub struct CacheManager { diff --git a/src/mito2/src/cache/file_cache.rs b/src/mito2/src/cache/file_cache.rs new file mode 100644 index 000000000000..25fd5d6d62bf --- /dev/null +++ b/src/mito2/src/cache/file_cache.rs @@ -0,0 +1,407 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! A cache for files. + +use std::time::Instant; + +use common_base::readable_size::ReadableSize; +use common_telemetry::{info, warn}; +use futures::{FutureExt, TryStreamExt}; +use moka::future::Cache; +use moka::notification::RemovalCause; +use object_store::util::{join_dir, join_path}; +use object_store::{ErrorKind, Metakey, ObjectStore, Reader}; +use snafu::ResultExt; +use store_api::storage::RegionId; + +use crate::cache::FILE_TYPE; +use crate::error::{OpenDalSnafu, Result}; +use crate::metrics::{CACHE_BYTES, CACHE_HIT, CACHE_MISS}; +use crate::sst::file::FileId; + +/// Subdirectory of cached files. +const FILE_DIR: &str = "files"; + +/// A file cache manages files on local store and evict files based +/// on size. +#[derive(Debug)] +pub(crate) struct FileCache { + /// Local store to cache files. + local_store: ObjectStore, + /// Cached file directory under cache home. + file_dir: String, + /// Index to track cached files. + /// + /// File id is enough to identity a file uniquely. + memory_index: Cache, +} + +impl FileCache { + /// Creates a new file cache. + pub(crate) fn new( + local_store: ObjectStore, + cache_home: String, + capacity: ReadableSize, + ) -> FileCache { + // Stores files under `cache_home/{FILE_DIR}`. + let file_dir = cache_file_dir(&cache_home); + let cache_store = local_store.clone(); + let cache_file_dir = file_dir.clone(); + let memory_index = Cache::builder() + .weigher(|_key, value: &IndexValue| -> u32 { + // We only measure space on local store. + value.file_size + }) + .max_capacity(capacity.as_bytes()) + .async_eviction_listener(move |key, value, cause| { + let store = cache_store.clone(); + let file_path = cache_file_path(&cache_file_dir, *key); + async move { + if let RemovalCause::Replaced = cause { + // The cache is replaced by another file. This is unexpected, we don't remove the same + // file but updates the metrics as the file is already replaced by users. + CACHE_BYTES.with_label_values(&[FILE_TYPE]).sub(value.file_size.into()); + warn!("Replace existing cache {} for region {} unexpectedly", file_path, key.0); + return; + } + + match store.delete(&file_path).await { + Ok(()) => { + CACHE_BYTES.with_label_values(&[FILE_TYPE]).sub(value.file_size.into()); + } + Err(e) => { + warn!(e; "Failed to delete cached file {} for region {}", file_path, key.0); + } + } + } + .boxed() + }) + .build(); + FileCache { + local_store, + file_dir, + memory_index, + } + } + + /// Puts a file into the cache index. + /// + /// The `WriteCache` should ensure the file is in the correct path. + pub(crate) async fn put(&self, key: IndexKey, value: IndexValue) { + CACHE_BYTES + .with_label_values(&[FILE_TYPE]) + .add(value.file_size.into()); + self.memory_index.insert(key, value).await; + } + + /// Reads a file from the cache. + pub(crate) async fn reader(&self, key: IndexKey) -> Option { + if !self.memory_index.contains_key(&key) { + CACHE_MISS.with_label_values(&[FILE_TYPE]).inc(); + return None; + } + + let file_path = self.cache_file_path(key); + match self.local_store.reader(&file_path).await { + Ok(reader) => { + CACHE_HIT.with_label_values(&[FILE_TYPE]).inc(); + Some(reader) + } + Err(e) => { + if e.kind() != ErrorKind::NotFound { + warn!("Failed to get file for key {:?}, err: {}", key, e); + } + // We removes the file from the index. + self.memory_index.remove(&key).await; + CACHE_MISS.with_label_values(&[FILE_TYPE]).inc(); + None + } + } + } + + /// Removes a file from the cache explicitly. + pub(crate) async fn remove(&self, key: IndexKey) { + let file_path = self.cache_file_path(key); + if let Err(e) = self.local_store.delete(&file_path).await { + warn!(e; "Failed to delete a cached file {}", file_path); + } + } + + /// Recovers the index from local store. + pub(crate) async fn recover(&self) -> Result<()> { + let now = Instant::now(); + + let mut lister = self + .local_store + .lister_with(&self.file_dir) + .metakey(Metakey::ContentLength) + .await + .context(OpenDalSnafu)?; + let (mut total_size, mut total_keys) = (0, 0); + while let Some(entry) = lister.try_next().await.context(OpenDalSnafu)? { + let meta = entry.metadata(); + if !meta.is_file() { + continue; + } + let Some(key) = parse_index_key(entry.name()) else { + continue; + }; + let file_size = meta.content_length() as u32; + self.memory_index + .insert(key, IndexValue { file_size }) + .await; + total_size += file_size; + total_keys += 1; + } + // The metrics is a signed int gauge so we can updates it finally. + CACHE_BYTES + .with_label_values(&[FILE_TYPE]) + .add(total_size.into()); + + info!( + "Recovered file cache, num_keys: {}, num_bytes: {}, cost: {:?}", + total_keys, + total_size, + now.elapsed() + ); + + Ok(()) + } + + /// Returns the cache file path for the key. + pub(crate) fn cache_file_path(&self, key: IndexKey) -> String { + cache_file_path(&self.file_dir, key) + } + + /// Returns the local store of the file cache. + pub(crate) fn local_store(&self) -> ObjectStore { + self.local_store.clone() + } +} + +/// Key of file cache index. +pub(crate) type IndexKey = (RegionId, FileId); + +/// An entity that describes the file in the file cache. +/// +/// It should only keep minimal information needed by the cache. +#[derive(Debug, Clone)] +pub(crate) struct IndexValue { + /// Size of the file in bytes. + file_size: u32, +} + +/// Returns the directory to store files. +fn cache_file_dir(cache_home: &str) -> String { + join_dir(cache_home, FILE_DIR) +} + +/// Generates the path to the cached file. +/// +/// The file name format is `{region_id}.{file_id}` +fn cache_file_path(cache_file_dir: &str, key: IndexKey) -> String { + join_path(cache_file_dir, &format!("{}.{}", key.0.as_u64(), key.1)) +} + +/// Parse index key from the file name. +fn parse_index_key(name: &str) -> Option { + let mut splited = name.splitn(2, '.'); + let region_id = splited.next().and_then(|s| { + let id = s.parse::().ok()?; + Some(RegionId::from_u64(id)) + })?; + let file_id = splited.next().and_then(|s| FileId::parse_str(s).ok())?; + + Some((region_id, file_id)) +} + +#[cfg(test)] +mod tests { + use common_test_util::temp_dir::create_temp_dir; + use futures::AsyncReadExt; + use object_store::services::Fs; + + use super::*; + + fn new_fs_store(path: &str) -> ObjectStore { + let mut builder = Fs::default(); + builder.root(path); + ObjectStore::new(builder).unwrap().finish() + } + + #[tokio::test] + async fn test_file_cache_basic() { + let dir = create_temp_dir(""); + let local_store = new_fs_store(dir.path().to_str().unwrap()); + let cache_home = "cache".to_string(); + + let cache = FileCache::new( + local_store.clone(), + cache_home.clone(), + ReadableSize::mb(10), + ); + let region_id = RegionId::new(2000, 0); + let file_id = FileId::random(); + let key = (region_id, file_id); + let file_path = cache.cache_file_path(key); + + // Get an empty file. + assert!(cache.reader(key).await.is_none()); + + // Write a file. + local_store + .write(&file_path, b"hello".as_slice()) + .await + .unwrap(); + // Add to the cache. + cache + .put((region_id, file_id), IndexValue { file_size: 5 }) + .await; + + // Read file content. + let mut reader = cache.reader(key).await.unwrap(); + let mut buf = String::new(); + reader.read_to_string(&mut buf).await.unwrap(); + assert_eq!("hello", buf); + + // Remove the file. + cache.remove(key).await; + assert!(cache.reader(key).await.is_none()); + + // Ensure all pending tasks of the moka cache is done before assertion. + cache.memory_index.run_pending_tasks().await; + + // The file also not exists. + assert!(!local_store.is_exist(&file_path).await.unwrap()); + } + + #[tokio::test] + async fn test_file_cache_file_removed() { + let dir = create_temp_dir(""); + let local_store = new_fs_store(dir.path().to_str().unwrap()); + let cache_home = "cache".to_string(); + + let cache = FileCache::new( + local_store.clone(), + cache_home.clone(), + ReadableSize::mb(10), + ); + let region_id = RegionId::new(2000, 0); + let file_id = FileId::random(); + let key = (region_id, file_id); + let file_path = cache.cache_file_path(key); + + // Write a file. + local_store + .write(&file_path, b"hello".as_slice()) + .await + .unwrap(); + // Add to the cache. + cache + .put((region_id, file_id), IndexValue { file_size: 5 }) + .await; + + // Remove the file but keep the index. + local_store.delete(&file_path).await.unwrap(); + + // Reader is none. + assert!(cache.reader(key).await.is_none()); + // Key is removed. + assert!(!cache.memory_index.contains_key(&key)); + } + + #[tokio::test] + async fn test_file_cache_recover() { + let dir = create_temp_dir(""); + let local_store = new_fs_store(dir.path().to_str().unwrap()); + let cache_home = "cache".to_string(); + let cache = FileCache::new( + local_store.clone(), + cache_home.clone(), + ReadableSize::mb(10), + ); + + let region_id = RegionId::new(2000, 0); + // Write N files. + let file_ids: Vec<_> = (0..10).map(|_| FileId::random()).collect(); + for (i, file_id) in file_ids.iter().enumerate() { + let key = (region_id, *file_id); + let file_path = cache.cache_file_path(key); + let bytes = i.to_string().into_bytes(); + local_store.write(&file_path, bytes.clone()).await.unwrap(); + + // Add to the cache. + cache + .put( + (region_id, *file_id), + IndexValue { + file_size: bytes.len() as u32, + }, + ) + .await; + } + + // Recover the cache. + let cache = FileCache::new( + local_store.clone(), + cache_home.clone(), + ReadableSize::mb(10), + ); + // No entry before recovery. + assert!(cache.reader((region_id, file_ids[0])).await.is_none()); + cache.recover().await.unwrap(); + + for (i, file_id) in file_ids.iter().enumerate() { + let key = (region_id, *file_id); + let mut reader = cache.reader(key).await.unwrap(); + let mut buf = String::new(); + reader.read_to_string(&mut buf).await.unwrap(); + assert_eq!(i.to_string(), buf); + } + } + + #[test] + fn test_cache_file_path() { + let file_id = FileId::parse_str("3368731b-a556-42b8-a5df-9c31ce155095").unwrap(); + assert_eq!( + "test_dir/5299989643269.3368731b-a556-42b8-a5df-9c31ce155095", + cache_file_path("test_dir", (RegionId::new(1234, 5), file_id)) + ); + assert_eq!( + "test_dir/5299989643269.3368731b-a556-42b8-a5df-9c31ce155095", + cache_file_path("test_dir/", (RegionId::new(1234, 5), file_id)) + ); + } + + #[test] + fn test_parse_file_name() { + let file_id = FileId::parse_str("3368731b-a556-42b8-a5df-9c31ce155095").unwrap(); + let region_id = RegionId::new(1234, 5); + assert_eq!( + (region_id, file_id), + parse_index_key("5299989643269.3368731b-a556-42b8-a5df-9c31ce155095").unwrap() + ); + assert!(parse_index_key("").is_none()); + assert!(parse_index_key(".").is_none()); + assert!(parse_index_key("5299989643269").is_none()); + assert!(parse_index_key("5299989643269.").is_none()); + assert!(parse_index_key(".5299989643269").is_none()); + assert!(parse_index_key("5299989643269.").is_none()); + assert!(parse_index_key("5299989643269.3368731b-a556-42b8-a5df").is_none()); + assert!( + parse_index_key("5299989643269.3368731b-a556-42b8-a5df-9c31ce155095.parquet").is_none() + ); + } +} From b9302e4f0d431be5c50a1e8c53b5aaf8bfe84db4 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Wed, 3 Jan 2024 13:14:40 +0800 Subject: [PATCH 21/21] feat(inverted_index): Add applier builder to convert Expr to Predicates (Part 2) (#3068) * feat(inverted_index.integration): Add applier builder to convert Expr to Predicates (Part 1) Signed-off-by: Zhenchi * feat(inverted_index.integration): Add applier builder to convert Expr to Predicates (Part 2) Signed-off-by: Zhenchi * test: add comparison unit tests Signed-off-by: Zhenchi * test: add eq_list unit tests Signed-off-by: Zhenchi * test: add in_list unit tests Signed-off-by: Zhenchi * test: add and unit tests Signed-off-by: Zhenchi * test: strip tests Signed-off-by: Zhenchi * fix: address comments Signed-off-by: Zhenchi --------- Signed-off-by: Zhenchi --- src/mito2/src/sst/index/applier/builder.rs | 133 ++++++-- .../sst/index/applier/builder/comparison.rs | 280 ++++++++++++++++ .../src/sst/index/applier/builder/eq_list.rs | 302 ++++++++++++++++++ .../src/sst/index/applier/builder/in_list.rs | 152 +++++++++ .../sst/index/applier/builder/regex_match.rs | 111 +++++++ 5 files changed, 947 insertions(+), 31 deletions(-) create mode 100644 src/mito2/src/sst/index/applier/builder/comparison.rs create mode 100644 src/mito2/src/sst/index/applier/builder/eq_list.rs create mode 100644 src/mito2/src/sst/index/applier/builder/in_list.rs create mode 100644 src/mito2/src/sst/index/applier/builder/regex_match.rs diff --git a/src/mito2/src/sst/index/applier/builder.rs b/src/mito2/src/sst/index/applier/builder.rs index 52af22effb18..240846a044c2 100644 --- a/src/mito2/src/sst/index/applier/builder.rs +++ b/src/mito2/src/sst/index/applier/builder.rs @@ -13,13 +13,10 @@ // limitations under the License. mod between; - -// TODO(zhongzc): This PR is too large. The following modules are coming soon. - -// mod comparison; -// mod eq_list; -// mod in_list; -// mod regex_match; +mod comparison; +mod eq_list; +mod in_list; +mod regex_match; use std::collections::HashMap; @@ -27,7 +24,7 @@ use api::v1::SemanticType; use common_query::logical_plan::Expr; use common_telemetry::warn; use datafusion_common::ScalarValue; -use datafusion_expr::Expr as DfExpr; +use datafusion_expr::{BinaryExpr, Expr as DfExpr, Operator}; use datatypes::data_type::ConcreteDataType; use datatypes::value::Value; use index::inverted_index::search::index_apply::PredicatesIndexApplier; @@ -99,23 +96,21 @@ impl<'a> SstIndexApplierBuilder<'a> { let res = match expr { DfExpr::Between(between) => self.collect_between(between), - // TODO(zhongzc): This PR is too large. The following arms are coming soon. - - // DfExpr::InList(in_list) => self.collect_inlist(in_list), - // DfExpr::BinaryExpr(BinaryExpr { left, op, right }) => match op { - // Operator::And => { - // self.traverse_and_collect(left); - // self.traverse_and_collect(right); - // Ok(()) - // } - // Operator::Or => self.collect_or_eq_list(left, right), - // Operator::Eq => self.collect_eq(left, right), - // Operator::Lt | Operator::LtEq | Operator::Gt | Operator::GtEq => { - // self.collect_comparison_expr(left, op, right) - // } - // Operator::RegexMatch => self.collect_regex_match(left, right), - // _ => Ok(()), - // }, + DfExpr::InList(in_list) => self.collect_inlist(in_list), + DfExpr::BinaryExpr(BinaryExpr { left, op, right }) => match op { + Operator::And => { + self.traverse_and_collect(left); + self.traverse_and_collect(right); + Ok(()) + } + Operator::Or => self.collect_or_eq_list(left, right), + Operator::Eq => self.collect_eq(left, right), + Operator::Lt | Operator::LtEq | Operator::Gt | Operator::GtEq => { + self.collect_comparison_expr(left, op, right) + } + Operator::RegexMatch => self.collect_regex_match(left, right), + _ => Ok(()), + }, // TODO(zhongzc): support more expressions, e.g. IsNull, IsNotNull, ... _ => Ok(()), @@ -180,8 +175,12 @@ impl<'a> SstIndexApplierBuilder<'a> { mod tests { use api::v1::SemanticType; use datafusion_common::Column; + use datafusion_expr::Between; use datatypes::data_type::ConcreteDataType; use datatypes::schema::ColumnSchema; + use index::inverted_index::search::predicate::{ + Bound, Range, RangePredicate, RegexMatchPredicate, + }; use object_store::services::Memory; use object_store::ObjectStore; use store_api::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataBuilder}; @@ -198,20 +197,25 @@ mod tests { column_id: 1, }) .push_column_metadata(ColumnMetadata { - column_schema: ColumnSchema::new("b", ConcreteDataType::string_datatype(), false), - semantic_type: SemanticType::Field, + column_schema: ColumnSchema::new("b", ConcreteDataType::int64_datatype(), false), + semantic_type: SemanticType::Tag, column_id: 2, }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new("c", ConcreteDataType::string_datatype(), false), + semantic_type: SemanticType::Field, + column_id: 3, + }) .push_column_metadata(ColumnMetadata { column_schema: ColumnSchema::new( - "c", + "d", ConcreteDataType::timestamp_millisecond_datatype(), false, ), semantic_type: SemanticType::Timestamp, - column_id: 3, + column_id: 4, }) - .primary_key(vec![1]); + .primary_key(vec![1, 2]); builder.build().unwrap() } @@ -226,13 +230,20 @@ mod tests { }) } - pub(crate) fn field_column() -> DfExpr { + pub(crate) fn tag_column2() -> DfExpr { DfExpr::Column(Column { relation: None, name: "b".to_string(), }) } + pub(crate) fn field_column() -> DfExpr { + DfExpr::Column(Column { + relation: None, + name: "c".to_string(), + }) + } + pub(crate) fn nonexistent_column() -> DfExpr { DfExpr::Column(Column { relation: None, @@ -258,4 +269,64 @@ mod tests { .unwrap(); bytes } + + pub(crate) fn encoded_int64(s: impl Into) -> Vec { + let mut bytes = vec![]; + IndexValueCodec::encode_value( + Value::from(s.into()).as_value_ref(), + &SortField::new(ConcreteDataType::int64_datatype()), + &mut bytes, + ) + .unwrap(); + bytes + } + + #[test] + fn test_collect_and_basic() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let expr = DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(tag_column()), + op: Operator::RegexMatch, + right: Box::new(string_lit("bar")), + })), + op: Operator::And, + right: Box::new(DfExpr::Between(Between { + expr: Box::new(tag_column2()), + negated: false, + low: Box::new(int64_lit(123)), + high: Box::new(int64_lit(456)), + })), + }); + + builder.traverse_and_collect(&expr); + let predicates = builder.output.get("a").unwrap(); + assert_eq!(predicates.len(), 1); + assert_eq!( + predicates[0], + Predicate::RegexMatch(RegexMatchPredicate { + pattern: "bar".to_string() + }) + ); + let predicates = builder.output.get("b").unwrap(); + assert_eq!(predicates.len(), 1); + assert_eq!( + predicates[0], + Predicate::Range(RangePredicate { + range: Range { + lower: Some(Bound { + inclusive: true, + value: encoded_int64(123), + }), + upper: Some(Bound { + inclusive: true, + value: encoded_int64(456), + }), + } + }) + ); + } } diff --git a/src/mito2/src/sst/index/applier/builder/comparison.rs b/src/mito2/src/sst/index/applier/builder/comparison.rs new file mode 100644 index 000000000000..e132c1c9281e --- /dev/null +++ b/src/mito2/src/sst/index/applier/builder/comparison.rs @@ -0,0 +1,280 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use datafusion_expr::{Expr as DfExpr, Operator}; +use index::inverted_index::search::predicate::{Bound, Predicate, Range, RangePredicate}; +use index::inverted_index::Bytes; + +use crate::error::Result; +use crate::sst::index::applier::builder::SstIndexApplierBuilder; + +impl<'a> SstIndexApplierBuilder<'a> { + /// Collects a comparison expression in the form of + /// `column < lit`, `column > lit`, `column <= lit`, `column >= lit`, + /// `lit < column`, `lit > column`, `lit <= column`, `lit >= column`. + pub(crate) fn collect_comparison_expr( + &mut self, + left: &DfExpr, + op: &Operator, + right: &DfExpr, + ) -> Result<()> { + match op { + Operator::Lt => { + if matches!(right, DfExpr::Column(_)) { + self.collect_column_gt_lit(right, left) + } else { + self.collect_column_lt_lit(left, right) + } + } + Operator::LtEq => { + if matches!(right, DfExpr::Column(_)) { + self.collect_column_ge_lit(right, left) + } else { + self.collect_column_le_lit(left, right) + } + } + Operator::Gt => { + if matches!(right, DfExpr::Column(_)) { + self.collect_column_lt_lit(right, left) + } else { + self.collect_column_gt_lit(left, right) + } + } + Operator::GtEq => { + if matches!(right, DfExpr::Column(_)) { + self.collect_column_le_lit(right, left) + } else { + self.collect_column_ge_lit(left, right) + } + } + _ => Ok(()), + } + } + + fn collect_column_lt_lit(&mut self, left: &DfExpr, right: &DfExpr) -> Result<()> { + self.collect_column_cmp_lit(left, right, |value| Range { + lower: None, + upper: Some(Bound { + inclusive: false, + value, + }), + }) + } + + fn collect_column_gt_lit(&mut self, left: &DfExpr, right: &DfExpr) -> Result<()> { + self.collect_column_cmp_lit(left, right, |value| Range { + lower: Some(Bound { + inclusive: false, + value, + }), + upper: None, + }) + } + + fn collect_column_le_lit(&mut self, left: &DfExpr, right: &DfExpr) -> Result<()> { + self.collect_column_cmp_lit(left, right, |value| Range { + lower: None, + upper: Some(Bound { + inclusive: true, + value, + }), + }) + } + + fn collect_column_ge_lit(&mut self, left: &DfExpr, right: &DfExpr) -> Result<()> { + self.collect_column_cmp_lit(left, right, |value| Range { + lower: Some(Bound { + inclusive: true, + value, + }), + upper: None, + }) + } + + fn collect_column_cmp_lit( + &mut self, + column: &DfExpr, + literal: &DfExpr, + range_builder: impl FnOnce(Bytes) -> Range, + ) -> Result<()> { + let Some(column_name) = Self::column_name(column) else { + return Ok(()); + }; + let Some(lit) = Self::nonnull_lit(literal) else { + return Ok(()); + }; + let Some(data_type) = self.tag_column_type(column_name)? else { + return Ok(()); + }; + + let predicate = Predicate::Range(RangePredicate { + range: range_builder(Self::encode_lit(lit, data_type)?), + }); + + self.add_predicate(column_name, predicate); + Ok(()) + } +} + +#[cfg(test)] +mod tests { + + use super::*; + use crate::error::Error; + use crate::sst::index::applier::builder::tests::{ + encoded_string, field_column, int64_lit, nonexistent_column, string_lit, tag_column, + test_object_store, test_region_metadata, + }; + + #[test] + fn test_collect_comparison_basic() { + let cases = [ + ( + (&tag_column(), &Operator::Lt, &string_lit("123")), + Range { + lower: None, + upper: Some(Bound { + inclusive: false, + value: encoded_string("123"), + }), + }, + ), + ( + (&string_lit("123"), &Operator::Lt, &tag_column()), + Range { + lower: Some(Bound { + inclusive: false, + value: encoded_string("123"), + }), + upper: None, + }, + ), + ( + (&tag_column(), &Operator::LtEq, &string_lit("123")), + Range { + lower: None, + upper: Some(Bound { + inclusive: true, + value: encoded_string("123"), + }), + }, + ), + ( + (&string_lit("123"), &Operator::LtEq, &tag_column()), + Range { + lower: Some(Bound { + inclusive: true, + value: encoded_string("123"), + }), + upper: None, + }, + ), + ( + (&tag_column(), &Operator::Gt, &string_lit("123")), + Range { + lower: Some(Bound { + inclusive: false, + value: encoded_string("123"), + }), + upper: None, + }, + ), + ( + (&string_lit("123"), &Operator::Gt, &tag_column()), + Range { + lower: None, + upper: Some(Bound { + inclusive: false, + value: encoded_string("123"), + }), + }, + ), + ( + (&tag_column(), &Operator::GtEq, &string_lit("123")), + Range { + lower: Some(Bound { + inclusive: true, + value: encoded_string("123"), + }), + upper: None, + }, + ), + ( + (&string_lit("123"), &Operator::GtEq, &tag_column()), + Range { + lower: None, + upper: Some(Bound { + inclusive: true, + value: encoded_string("123"), + }), + }, + ), + ]; + + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + for ((left, op, right), _) in &cases { + builder.collect_comparison_expr(left, op, right).unwrap(); + } + + let predicates = builder.output.get("a").unwrap(); + assert_eq!(predicates.len(), cases.len()); + for ((_, expected), actual) in cases.into_iter().zip(predicates) { + assert_eq!( + actual, + &Predicate::Range(RangePredicate { range: expected }) + ); + } + } + + #[test] + fn test_collect_comparison_type_mismatch() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let res = builder.collect_comparison_expr(&tag_column(), &Operator::Lt, &int64_lit(10)); + assert!(matches!(res, Err(Error::FieldTypeMismatch { .. }))); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_comparison_field_column() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + builder + .collect_comparison_expr(&field_column(), &Operator::Lt, &string_lit("abc")) + .unwrap(); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_comparison_nonexistent_column() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let res = builder.collect_comparison_expr( + &nonexistent_column(), + &Operator::Lt, + &string_lit("abc"), + ); + assert!(matches!(res, Err(Error::ColumnNotFound { .. }))); + assert!(builder.output.is_empty()); + } +} diff --git a/src/mito2/src/sst/index/applier/builder/eq_list.rs b/src/mito2/src/sst/index/applier/builder/eq_list.rs new file mode 100644 index 000000000000..07e74e012db0 --- /dev/null +++ b/src/mito2/src/sst/index/applier/builder/eq_list.rs @@ -0,0 +1,302 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashSet; + +use datafusion_expr::{BinaryExpr, Expr as DfExpr, Operator}; +use datatypes::data_type::ConcreteDataType; +use index::inverted_index::search::predicate::{InListPredicate, Predicate}; +use index::inverted_index::Bytes; + +use crate::error::Result; +use crate::sst::index::applier::builder::SstIndexApplierBuilder; + +impl<'a> SstIndexApplierBuilder<'a> { + /// Collects an eq expression in the form of `column = lit`. + pub(crate) fn collect_eq(&mut self, left: &DfExpr, right: &DfExpr) -> Result<()> { + let Some(column_name) = Self::column_name(left).or_else(|| Self::column_name(right)) else { + return Ok(()); + }; + let Some(lit) = Self::nonnull_lit(right).or_else(|| Self::nonnull_lit(left)) else { + return Ok(()); + }; + let Some(data_type) = self.tag_column_type(column_name)? else { + return Ok(()); + }; + + let predicate = Predicate::InList(InListPredicate { + list: HashSet::from_iter([Self::encode_lit(lit, data_type)?]), + }); + self.add_predicate(column_name, predicate); + Ok(()) + } + + /// Collects eq list in the form of `column = lit OR column = lit OR ...`. + pub(crate) fn collect_or_eq_list(&mut self, eq_expr: &DfExpr, or_list: &DfExpr) -> Result<()> { + let DfExpr::BinaryExpr(BinaryExpr { + left, + op: Operator::Eq, + right, + }) = eq_expr + else { + return Ok(()); + }; + + let Some(column_name) = Self::column_name(left).or_else(|| Self::column_name(right)) else { + return Ok(()); + }; + let Some(lit) = Self::nonnull_lit(right).or_else(|| Self::nonnull_lit(left)) else { + return Ok(()); + }; + let Some(data_type) = self.tag_column_type(column_name)? else { + return Ok(()); + }; + + let bytes = Self::encode_lit(lit, data_type.clone())?; + let mut inlist = HashSet::from_iter([bytes]); + + if Self::collect_eq_list_inner(column_name, &data_type, or_list, &mut inlist)? { + let predicate = Predicate::InList(InListPredicate { list: inlist }); + self.add_predicate(column_name, predicate); + } + + Ok(()) + } + + /// Recursively collects eq list. + /// + /// Returns false if the expression doesn't match the form then + /// caller can safely ignore the expression. + fn collect_eq_list_inner( + column_name: &str, + data_type: &ConcreteDataType, + expr: &DfExpr, + inlist: &mut HashSet, + ) -> Result { + let DfExpr::BinaryExpr(BinaryExpr { + left, + op: op @ (Operator::Eq | Operator::Or), + right, + }) = expr + else { + return Ok(false); + }; + + if op == &Operator::Or { + let r = Self::collect_eq_list_inner(column_name, data_type, left, inlist)? + .then(|| Self::collect_eq_list_inner(column_name, data_type, right, inlist)) + .transpose()? + .unwrap_or(false); + return Ok(r); + } + + if op == &Operator::Eq { + let Some(name) = Self::column_name(left).or_else(|| Self::column_name(right)) else { + return Ok(false); + }; + if column_name != name { + return Ok(false); + } + let Some(lit) = Self::nonnull_lit(right).or_else(|| Self::nonnull_lit(left)) else { + return Ok(false); + }; + + inlist.insert(Self::encode_lit(lit, data_type.clone())?); + return Ok(true); + } + + Ok(false) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::error::Error; + use crate::sst::index::applier::builder::tests::{ + encoded_string, field_column, int64_lit, nonexistent_column, string_lit, tag_column, + tag_column2, test_object_store, test_region_metadata, + }; + + #[test] + fn test_collect_eq_basic() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + builder + .collect_eq(&tag_column(), &string_lit("foo")) + .unwrap(); + builder + .collect_eq(&string_lit("bar"), &tag_column()) + .unwrap(); + + let predicates = builder.output.get("a").unwrap(); + assert_eq!(predicates.len(), 2); + assert_eq!( + predicates[0], + Predicate::InList(InListPredicate { + list: HashSet::from_iter([encoded_string("foo")]) + }) + ); + assert_eq!( + predicates[1], + Predicate::InList(InListPredicate { + list: HashSet::from_iter([encoded_string("bar")]) + }) + ); + } + + #[test] + fn test_collect_eq_field_column() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + builder + .collect_eq(&field_column(), &string_lit("abc")) + .unwrap(); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_eq_nonexistent_column() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let res = builder.collect_eq(&nonexistent_column(), &string_lit("abc")); + assert!(matches!(res, Err(Error::ColumnNotFound { .. }))); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_eq_type_mismatch() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let res = builder.collect_eq(&tag_column(), &int64_lit(1)); + assert!(matches!(res, Err(Error::FieldTypeMismatch { .. }))); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_or_eq_list_basic() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let eq_expr = DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(tag_column()), + op: Operator::Eq, + right: Box::new(string_lit("abc")), + }); + let or_eq_list = DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(tag_column()), + op: Operator::Eq, + right: Box::new(string_lit("foo")), + })), + op: Operator::Or, + right: Box::new(DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(tag_column()), + op: Operator::Eq, + right: Box::new(string_lit("bar")), + })), + op: Operator::Or, + right: Box::new(DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(tag_column()), + op: Operator::Eq, + right: Box::new(string_lit("baz")), + })), + })), + }); + + builder.collect_or_eq_list(&eq_expr, &or_eq_list).unwrap(); + + let predicates = builder.output.get("a").unwrap(); + assert_eq!(predicates.len(), 1); + assert_eq!( + predicates[0], + Predicate::InList(InListPredicate { + list: HashSet::from_iter([ + encoded_string("abc"), + encoded_string("foo"), + encoded_string("bar"), + encoded_string("baz") + ]) + }) + ); + } + + #[test] + fn test_collect_or_eq_list_invalid_op() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let eq_expr = DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(tag_column()), + op: Operator::Eq, + right: Box::new(string_lit("abc")), + }); + let or_eq_list = DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(tag_column()), + op: Operator::Eq, + right: Box::new(string_lit("foo")), + })), + op: Operator::Or, + right: Box::new(DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(tag_column()), + op: Operator::Gt, // invalid op + right: Box::new(string_lit("foo")), + })), + }); + + builder.collect_or_eq_list(&eq_expr, &or_eq_list).unwrap(); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_or_eq_list_multiple_columns() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let eq_expr = DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(tag_column()), + op: Operator::Eq, + right: Box::new(string_lit("abc")), + }); + let or_eq_list = DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(tag_column()), + op: Operator::Eq, + right: Box::new(string_lit("foo")), + })), + op: Operator::Or, + right: Box::new(DfExpr::BinaryExpr(BinaryExpr { + left: Box::new(tag_column2()), // different column + op: Operator::Eq, + right: Box::new(string_lit("bar")), + })), + }); + + builder.collect_or_eq_list(&eq_expr, &or_eq_list).unwrap(); + assert!(builder.output.is_empty()); + } +} diff --git a/src/mito2/src/sst/index/applier/builder/in_list.rs b/src/mito2/src/sst/index/applier/builder/in_list.rs new file mode 100644 index 000000000000..cfb2b8738f62 --- /dev/null +++ b/src/mito2/src/sst/index/applier/builder/in_list.rs @@ -0,0 +1,152 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashSet; + +use datafusion_expr::expr::InList; +use index::inverted_index::search::predicate::{InListPredicate, Predicate}; + +use crate::error::Result; +use crate::sst::index::applier::builder::SstIndexApplierBuilder; + +impl<'a> SstIndexApplierBuilder<'a> { + /// Collects an in list expression in the form of `column IN (lit, lit, ...)`. + pub(crate) fn collect_inlist(&mut self, inlist: &InList) -> Result<()> { + if inlist.negated { + return Ok(()); + } + let Some(column_name) = Self::column_name(&inlist.expr) else { + return Ok(()); + }; + let Some(data_type) = self.tag_column_type(column_name)? else { + return Ok(()); + }; + + let mut predicate = InListPredicate { + list: HashSet::with_capacity(inlist.list.len()), + }; + for lit in &inlist.list { + let Some(lit) = Self::nonnull_lit(lit) else { + return Ok(()); + }; + + predicate + .list + .insert(Self::encode_lit(lit, data_type.clone())?); + } + + self.add_predicate(column_name, Predicate::InList(predicate)); + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::error::Error; + use crate::sst::index::applier::builder::tests::{ + encoded_string, field_column, int64_lit, nonexistent_column, string_lit, tag_column, + test_object_store, test_region_metadata, + }; + + #[test] + fn test_collect_in_list_basic() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let in_list = InList { + expr: Box::new(tag_column()), + list: vec![string_lit("foo"), string_lit("bar")], + negated: false, + }; + + builder.collect_inlist(&in_list).unwrap(); + + let predicates = builder.output.get("a").unwrap(); + assert_eq!(predicates.len(), 1); + assert_eq!( + predicates[0], + Predicate::InList(InListPredicate { + list: HashSet::from_iter([encoded_string("foo"), encoded_string("bar")]) + }) + ); + } + + #[test] + fn test_collect_in_list_negated() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let in_list = InList { + expr: Box::new(tag_column()), + list: vec![string_lit("foo"), string_lit("bar")], + negated: true, + }; + + builder.collect_inlist(&in_list).unwrap(); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_in_list_field_column() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let in_list = InList { + expr: Box::new(field_column()), + list: vec![string_lit("foo"), string_lit("bar")], + negated: false, + }; + + builder.collect_inlist(&in_list).unwrap(); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_in_list_type_mismatch() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let in_list = InList { + expr: Box::new(tag_column()), + list: vec![int64_lit(123), int64_lit(456)], + negated: false, + }; + + let res = builder.collect_inlist(&in_list); + assert!(matches!(res, Err(Error::FieldTypeMismatch { .. }))); + assert!(builder.output.is_empty()); + } + + #[test] + fn test_collect_in_list_nonexistent_column() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let in_list = InList { + expr: Box::new(nonexistent_column()), + list: vec![string_lit("foo"), string_lit("bar")], + negated: false, + }; + + let res = builder.collect_inlist(&in_list); + assert!(matches!(res, Err(Error::ColumnNotFound { .. }))); + assert!(builder.output.is_empty()); + } +} diff --git a/src/mito2/src/sst/index/applier/builder/regex_match.rs b/src/mito2/src/sst/index/applier/builder/regex_match.rs new file mode 100644 index 000000000000..1aa1cd9d95c1 --- /dev/null +++ b/src/mito2/src/sst/index/applier/builder/regex_match.rs @@ -0,0 +1,111 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use datafusion_common::ScalarValue; +use datafusion_expr::Expr as DfExpr; +use index::inverted_index::search::predicate::{Predicate, RegexMatchPredicate}; + +use crate::error::Result; +use crate::sst::index::applier::builder::SstIndexApplierBuilder; + +impl<'a> SstIndexApplierBuilder<'a> { + /// Collects a regex match expression in the form of `column ~ pattern`. + pub(crate) fn collect_regex_match(&mut self, column: &DfExpr, pattern: &DfExpr) -> Result<()> { + let Some(column_name) = Self::column_name(column) else { + return Ok(()); + }; + let Some(data_type) = self.tag_column_type(column_name)? else { + return Ok(()); + }; + if !data_type.is_string() { + return Ok(()); + } + let DfExpr::Literal(ScalarValue::Utf8(Some(pattern))) = pattern else { + return Ok(()); + }; + + let predicate = Predicate::RegexMatch(RegexMatchPredicate { + pattern: pattern.clone(), + }); + self.add_predicate(column_name, predicate); + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::error::Error; + use crate::sst::index::applier::builder::tests::{ + field_column, int64_lit, nonexistent_column, string_lit, tag_column, test_object_store, + test_region_metadata, + }; + + #[test] + fn test_regex_match_basic() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + builder + .collect_regex_match(&tag_column(), &string_lit("abc")) + .unwrap(); + + let predicates = builder.output.get("a").unwrap(); + assert_eq!(predicates.len(), 1); + assert_eq!( + predicates[0], + Predicate::RegexMatch(RegexMatchPredicate { + pattern: "abc".to_string() + }) + ); + } + + #[test] + fn test_regex_match_field_column() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + builder + .collect_regex_match(&field_column(), &string_lit("abc")) + .unwrap(); + + assert!(builder.output.is_empty()); + } + + #[test] + fn test_regex_match_type_mismatch() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + builder + .collect_regex_match(&tag_column(), &int64_lit(123)) + .unwrap(); + + assert!(builder.output.is_empty()); + } + + #[test] + fn test_regex_match_type_nonexist_column() { + let metadata = test_region_metadata(); + let mut builder = + SstIndexApplierBuilder::new("test".to_string(), test_object_store(), &metadata); + + let res = builder.collect_regex_match(&nonexistent_column(), &string_lit("abc")); + assert!(matches!(res, Err(Error::ColumnNotFound { .. }))); + assert!(builder.output.is_empty()); + } +}