From 2d6f63a504b5da77134feed4e843e6131bd2c748 Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Thu, 19 Dec 2024 16:24:04 +0800 Subject: [PATCH 01/12] feat: show flow's mem usage in INFORMATION_SCHEMA.FLOWS (#4890) * feat: add flow mem size to sys table * chore: rm dup def * chore: remove unused variant * chore: minor refactor * refactor: per review --- Cargo.lock | 128 ++++++++++++++ src/catalog/src/error.rs | 8 + src/catalog/src/information_extension.rs | 9 + .../src/system_schema/information_schema.rs | 9 + .../system_schema/information_schema/flows.rs | 35 +++- src/cli/src/repl.rs | 2 +- src/cmd/src/standalone.rs | 33 +++- src/common/meta/src/key.rs | 4 +- src/common/meta/src/key/flow.rs | 9 + src/common/meta/src/key/flow/flow_state.rs | 162 ++++++++++++++++++ src/common/meta/src/kv_backend.rs | 7 +- src/common/meta/src/kv_backend/memory.rs | 8 +- src/flow/Cargo.toml | 2 + src/flow/src/adapter.rs | 32 ++++ src/flow/src/adapter/stat.rs | 40 +++++ src/flow/src/adapter/worker.rs | 30 ++++ src/flow/src/compute/state.rs | 5 + src/flow/src/heartbeat.rs | 49 +++++- src/flow/src/repr.rs | 8 + src/flow/src/server.rs | 17 +- src/flow/src/utils.rs | 139 ++++++++++++++- src/meta-client/src/client.rs | 15 +- src/meta-client/src/client/cluster.rs | 76 +++++++- src/meta-client/src/error.rs | 20 ++- src/meta-srv/src/error.rs | 10 +- src/meta-srv/src/handler.rs | 14 ++ .../src/handler/flow_state_handler.rs | 58 +++++++ src/meta-srv/src/metasrv/builder.rs | 7 + src/meta-srv/src/service/store/cached_kv.rs | 4 + .../standalone/common/flow/flow_basic.result | 12 ++ .../standalone/common/flow/flow_basic.sql | 6 + .../common/system/information_schema.result | 17 +- 32 files changed, 942 insertions(+), 33 deletions(-) create mode 100644 src/common/meta/src/key/flow/flow_state.rs create mode 100644 src/flow/src/adapter/stat.rs create mode 100644 src/meta-srv/src/handler/flow_state_handler.rs diff --git a/Cargo.lock b/Cargo.lock index fa8ba34d1a3b..c23acf60636d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -730,6 +730,36 @@ version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1505bd5d3d116872e7271a6d4e16d81d0c8570876c8de68093a09ac269d8aac0" +[[package]] +name = "attribute-derive" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1800e974930e9079c965b9ffbcb6667a40401063a26396c7b4f15edc92da690" +dependencies = [ + "attribute-derive-macro", + "derive-where", + "manyhow", + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "attribute-derive-macro" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5d908eb786ef94296bff86f90130b3b748b49401dc81fd2bb8b3dccd44cfacbd" +dependencies = [ + "collection_literals", + "interpolator", + "manyhow", + "proc-macro-utils", + "proc-macro2", + "quote", + "quote-use", + "syn 2.0.90", +] + [[package]] name = "atty" version = "0.2.14" @@ -1845,6 +1875,12 @@ dependencies = [ "tracing-appender", ] +[[package]] +name = "collection_literals" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "186dce98367766de751c42c4f03970fc60fc012296e706ccbb9d5df9b6c1e271" + [[package]] name = "colorchoice" version = "1.0.2" @@ -3346,6 +3382,17 @@ dependencies = [ "syn 2.0.90", ] +[[package]] +name = "derive-where" +version = "1.2.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62d671cc41a825ebabc75757b62d3d168c577f9149b2d49ece1dad1f72119d25" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + [[package]] name = "derive_arbitrary" version = "1.3.2" @@ -4011,6 +4058,8 @@ dependencies = [ "enum-as-inner", "enum_dispatch", "futures", + "get-size-derive2", + "get-size2", "greptime-proto", "hydroflow", "itertools 0.10.5", @@ -4415,6 +4464,23 @@ dependencies = [ "libm", ] +[[package]] +name = "get-size-derive2" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fd26d3a97ea14d289c8b54180243ecfe465f3fa9c279a6336d7a003698fc39d" +dependencies = [ + "attribute-derive", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "get-size2" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "159c430715e540d2198fa981d39cd45563ccc60900de187f5b152b33b1cb408e" + [[package]] name = "gethostname" version = "0.2.3" @@ -5346,6 +5412,12 @@ version = "4.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0d762194228a2f1c11063e46e32e5acb96e66e906382b9eb5441f2e0504bbd5a" +[[package]] +name = "interpolator" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "71dd52191aae121e8611f1e8dc3e324dd0dd1dee1e6dd91d10ee07a3cfb4d9d8" + [[package]] name = "inventory" version = "0.3.15" @@ -6244,6 +6316,29 @@ dependencies = [ "libc", ] +[[package]] +name = "manyhow" +version = "0.11.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b33efb3ca6d3b07393750d4030418d594ab1139cee518f0dc88db70fec873587" +dependencies = [ + "manyhow-macros", + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "manyhow-macros" +version = "0.11.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46fce34d199b78b6e6073abf984c9cf5fd3e9330145a93ee0738a7443e371495" +dependencies = [ + "proc-macro-utils", + "proc-macro2", + "quote", +] + [[package]] name = "maplit" version = "1.0.2" @@ -8528,6 +8623,17 @@ dependencies = [ "version_check", ] +[[package]] +name = "proc-macro-utils" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eeaf08a13de400bc215877b5bdc088f241b12eb42f0a548d3390dc1c56bb7071" +dependencies = [ + "proc-macro2", + "quote", + "smallvec", +] + [[package]] name = "proc-macro2" version = "1.0.92" @@ -9107,6 +9213,28 @@ dependencies = [ "proc-macro2", ] +[[package]] +name = "quote-use" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9619db1197b497a36178cfc736dc96b271fe918875fbf1344c436a7e93d0321e" +dependencies = [ + "quote", + "quote-use-macros", +] + +[[package]] +name = "quote-use-macros" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "82ebfb7faafadc06a7ab141a6f67bcfb24cb8beb158c6fe933f2f035afa99f35" +dependencies = [ + "proc-macro-utils", + "proc-macro2", + "quote", + "syn 2.0.90", +] + [[package]] name = "radium" version = "0.7.0" diff --git a/src/catalog/src/error.rs b/src/catalog/src/error.rs index c7e6f8b55c01..4980a8178eb6 100644 --- a/src/catalog/src/error.rs +++ b/src/catalog/src/error.rs @@ -64,6 +64,13 @@ pub enum Error { source: BoxedError, }, + #[snafu(display("Failed to list flow stats"))] + ListFlowStats { + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + #[snafu(display("Failed to list flows in catalog {catalog}"))] ListFlows { #[snafu(implicit)] @@ -326,6 +333,7 @@ impl ErrorExt for Error { | Error::ListSchemas { source, .. } | Error::ListTables { source, .. } | Error::ListFlows { source, .. } + | Error::ListFlowStats { source, .. } | Error::ListProcedures { source, .. } | Error::ListRegionStats { source, .. } | Error::ConvertProtoData { source, .. } => source.status_code(), diff --git a/src/catalog/src/information_extension.rs b/src/catalog/src/information_extension.rs index 55764557a326..4d829ae01ae3 100644 --- a/src/catalog/src/information_extension.rs +++ b/src/catalog/src/information_extension.rs @@ -17,6 +17,7 @@ use common_error::ext::BoxedError; use common_meta::cluster::{ClusterInfo, NodeInfo}; use common_meta::datanode::RegionStat; use common_meta::ddl::{ExecutorContext, ProcedureExecutor}; +use common_meta::key::flow::flow_state::FlowStat; use common_meta::rpc::procedure; use common_procedure::{ProcedureInfo, ProcedureState}; use meta_client::MetaClientRef; @@ -89,4 +90,12 @@ impl InformationExtension for DistributedInformationExtension { .map_err(BoxedError::new) .context(error::ListRegionStatsSnafu) } + + async fn flow_stats(&self) -> std::result::Result, Self::Error> { + self.meta_client + .list_flow_stats() + .await + .map_err(BoxedError::new) + .context(crate::error::ListFlowStatsSnafu) + } } diff --git a/src/catalog/src/system_schema/information_schema.rs b/src/catalog/src/system_schema/information_schema.rs index 4101887cb443..6b3231cc080a 100644 --- a/src/catalog/src/system_schema/information_schema.rs +++ b/src/catalog/src/system_schema/information_schema.rs @@ -35,6 +35,7 @@ use common_catalog::consts::{self, DEFAULT_CATALOG_NAME, INFORMATION_SCHEMA_NAME use common_error::ext::ErrorExt; use common_meta::cluster::NodeInfo; use common_meta::datanode::RegionStat; +use common_meta::key::flow::flow_state::FlowStat; use common_meta::key::flow::FlowMetadataManager; use common_procedure::ProcedureInfo; use common_recordbatch::SendableRecordBatchStream; @@ -192,6 +193,7 @@ impl SystemSchemaProviderInner for InformationSchemaProvider { )) as _), FLOWS => Some(Arc::new(InformationSchemaFlows::new( self.catalog_name.clone(), + self.catalog_manager.clone(), self.flow_metadata_manager.clone(), )) as _), PROCEDURE_INFO => Some( @@ -338,6 +340,9 @@ pub trait InformationExtension { /// Gets the region statistics. async fn region_stats(&self) -> std::result::Result, Self::Error>; + + /// Get the flow statistics. If no flownode is available, return `None`. + async fn flow_stats(&self) -> std::result::Result, Self::Error>; } pub struct NoopInformationExtension; @@ -357,4 +362,8 @@ impl InformationExtension for NoopInformationExtension { async fn region_stats(&self) -> std::result::Result, Self::Error> { Ok(vec![]) } + + async fn flow_stats(&self) -> std::result::Result, Self::Error> { + Ok(None) + } } diff --git a/src/catalog/src/system_schema/information_schema/flows.rs b/src/catalog/src/system_schema/information_schema/flows.rs index 15a4205ae2af..5d35cfbbe431 100644 --- a/src/catalog/src/system_schema/information_schema/flows.rs +++ b/src/catalog/src/system_schema/information_schema/flows.rs @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; +use std::sync::{Arc, Weak}; use common_catalog::consts::INFORMATION_SCHEMA_FLOW_TABLE_ID; use common_error::ext::BoxedError; use common_meta::key::flow::flow_info::FlowInfoValue; +use common_meta::key::flow::flow_state::FlowStat; use common_meta::key::flow::FlowMetadataManager; use common_meta::key::FlowId; use common_recordbatch::adapter::RecordBatchStreamAdapter; @@ -28,7 +29,9 @@ use datatypes::prelude::ConcreteDataType as CDT; use datatypes::scalars::ScalarVectorBuilder; use datatypes::schema::{ColumnSchema, Schema, SchemaRef}; use datatypes::value::Value; -use datatypes::vectors::{Int64VectorBuilder, StringVectorBuilder, UInt32VectorBuilder, VectorRef}; +use datatypes::vectors::{ + Int64VectorBuilder, StringVectorBuilder, UInt32VectorBuilder, UInt64VectorBuilder, VectorRef, +}; use futures::TryStreamExt; use snafu::{OptionExt, ResultExt}; use store_api::storage::{ScanRequest, TableId}; @@ -38,6 +41,8 @@ use crate::error::{ }; use crate::information_schema::{Predicates, FLOWS}; use crate::system_schema::information_schema::InformationTable; +use crate::system_schema::utils; +use crate::CatalogManager; const INIT_CAPACITY: usize = 42; @@ -45,6 +50,7 @@ const INIT_CAPACITY: usize = 42; // pk is (flow_name, flow_id, table_catalog) pub const FLOW_NAME: &str = "flow_name"; pub const FLOW_ID: &str = "flow_id"; +pub const STATE_SIZE: &str = "state_size"; pub const TABLE_CATALOG: &str = "table_catalog"; pub const FLOW_DEFINITION: &str = "flow_definition"; pub const COMMENT: &str = "comment"; @@ -55,20 +61,24 @@ pub const FLOWNODE_IDS: &str = "flownode_ids"; pub const OPTIONS: &str = "options"; /// The `information_schema.flows` to provides information about flows in databases. +/// pub(super) struct InformationSchemaFlows { schema: SchemaRef, catalog_name: String, + catalog_manager: Weak, flow_metadata_manager: Arc, } impl InformationSchemaFlows { pub(super) fn new( catalog_name: String, + catalog_manager: Weak, flow_metadata_manager: Arc, ) -> Self { Self { schema: Self::schema(), catalog_name, + catalog_manager, flow_metadata_manager, } } @@ -80,6 +90,7 @@ impl InformationSchemaFlows { vec![ (FLOW_NAME, CDT::string_datatype(), false), (FLOW_ID, CDT::uint32_datatype(), false), + (STATE_SIZE, CDT::uint64_datatype(), true), (TABLE_CATALOG, CDT::string_datatype(), false), (FLOW_DEFINITION, CDT::string_datatype(), false), (COMMENT, CDT::string_datatype(), true), @@ -99,6 +110,7 @@ impl InformationSchemaFlows { InformationSchemaFlowsBuilder::new( self.schema.clone(), self.catalog_name.clone(), + self.catalog_manager.clone(), &self.flow_metadata_manager, ) } @@ -144,10 +156,12 @@ impl InformationTable for InformationSchemaFlows { struct InformationSchemaFlowsBuilder { schema: SchemaRef, catalog_name: String, + catalog_manager: Weak, flow_metadata_manager: Arc, flow_names: StringVectorBuilder, flow_ids: UInt32VectorBuilder, + state_sizes: UInt64VectorBuilder, table_catalogs: StringVectorBuilder, raw_sqls: StringVectorBuilder, comments: StringVectorBuilder, @@ -162,15 +176,18 @@ impl InformationSchemaFlowsBuilder { fn new( schema: SchemaRef, catalog_name: String, + catalog_manager: Weak, flow_metadata_manager: &Arc, ) -> Self { Self { schema, catalog_name, + catalog_manager, flow_metadata_manager: flow_metadata_manager.clone(), flow_names: StringVectorBuilder::with_capacity(INIT_CAPACITY), flow_ids: UInt32VectorBuilder::with_capacity(INIT_CAPACITY), + state_sizes: UInt64VectorBuilder::with_capacity(INIT_CAPACITY), table_catalogs: StringVectorBuilder::with_capacity(INIT_CAPACITY), raw_sqls: StringVectorBuilder::with_capacity(INIT_CAPACITY), comments: StringVectorBuilder::with_capacity(INIT_CAPACITY), @@ -195,6 +212,11 @@ impl InformationSchemaFlowsBuilder { .flow_names(&catalog_name) .await; + let flow_stat = { + let information_extension = utils::information_extension(&self.catalog_manager)?; + information_extension.flow_stats().await? + }; + while let Some((flow_name, flow_id)) = stream .try_next() .await @@ -213,7 +235,7 @@ impl InformationSchemaFlowsBuilder { catalog_name: catalog_name.to_string(), flow_name: flow_name.to_string(), })?; - self.add_flow(&predicates, flow_id.flow_id(), flow_info)?; + self.add_flow(&predicates, flow_id.flow_id(), flow_info, &flow_stat)?; } self.finish() @@ -224,6 +246,7 @@ impl InformationSchemaFlowsBuilder { predicates: &Predicates, flow_id: FlowId, flow_info: FlowInfoValue, + flow_stat: &Option, ) -> Result<()> { let row = [ (FLOW_NAME, &Value::from(flow_info.flow_name().to_string())), @@ -238,6 +261,11 @@ impl InformationSchemaFlowsBuilder { } self.flow_names.push(Some(flow_info.flow_name())); self.flow_ids.push(Some(flow_id)); + self.state_sizes.push( + flow_stat + .as_ref() + .and_then(|state| state.state_size.get(&flow_id).map(|v| *v as u64)), + ); self.table_catalogs.push(Some(flow_info.catalog_name())); self.raw_sqls.push(Some(flow_info.raw_sql())); self.comments.push(Some(flow_info.comment())); @@ -270,6 +298,7 @@ impl InformationSchemaFlowsBuilder { let columns: Vec = vec![ Arc::new(self.flow_names.finish()), Arc::new(self.flow_ids.finish()), + Arc::new(self.state_sizes.finish()), Arc::new(self.table_catalogs.finish()), Arc::new(self.raw_sqls.finish()), Arc::new(self.comments.finish()), diff --git a/src/cli/src/repl.rs b/src/cli/src/repl.rs index 4c2ef8ffe396..8b5e3aa389a2 100644 --- a/src/cli/src/repl.rs +++ b/src/cli/src/repl.rs @@ -34,7 +34,7 @@ use common_query::Output; use common_recordbatch::RecordBatches; use common_telemetry::debug; use either::Either; -use meta_client::client::MetaClientBuilder; +use meta_client::client::{ClusterKvBackend, MetaClientBuilder}; use query::datafusion::DatafusionQueryEngine; use query::parser::QueryLanguageParser; use query::query_engine::{DefaultSerializer, QueryEngineState}; diff --git a/src/cmd/src/standalone.rs b/src/cmd/src/standalone.rs index d7e816166b03..8490e14147b2 100644 --- a/src/cmd/src/standalone.rs +++ b/src/cmd/src/standalone.rs @@ -34,6 +34,7 @@ use common_meta::ddl::flow_meta::{FlowMetadataAllocator, FlowMetadataAllocatorRe use common_meta::ddl::table_meta::{TableMetadataAllocator, TableMetadataAllocatorRef}; use common_meta::ddl::{DdlContext, NoopRegionFailureDetectorControl, ProcedureExecutorRef}; use common_meta::ddl_manager::DdlManager; +use common_meta::key::flow::flow_state::FlowStat; use common_meta::key::flow::{FlowMetadataManager, FlowMetadataManagerRef}; use common_meta::key::{TableMetadataManager, TableMetadataManagerRef}; use common_meta::kv_backend::KvBackendRef; @@ -70,7 +71,7 @@ use servers::http::HttpOptions; use servers::tls::{TlsMode, TlsOption}; use servers::Mode; use snafu::ResultExt; -use tokio::sync::broadcast; +use tokio::sync::{broadcast, RwLock}; use tracing_appender::non_blocking::WorkerGuard; use crate::error::{ @@ -507,7 +508,7 @@ impl StartCommand { procedure_manager.clone(), )); let catalog_manager = KvBackendCatalogManager::new( - information_extension, + information_extension.clone(), kv_backend.clone(), layered_cache_registry.clone(), Some(procedure_manager.clone()), @@ -532,6 +533,14 @@ impl StartCommand { .context(OtherSnafu)?, ); + // set the ref to query for the local flow state + { + let flow_worker_manager = flownode.flow_worker_manager(); + information_extension + .set_flow_worker_manager(flow_worker_manager.clone()) + .await; + } + let node_manager = Arc::new(StandaloneDatanodeManager { region_server: datanode.region_server(), flow_server: flownode.flow_worker_manager(), @@ -669,6 +678,7 @@ pub struct StandaloneInformationExtension { region_server: RegionServer, procedure_manager: ProcedureManagerRef, start_time_ms: u64, + flow_worker_manager: RwLock>>, } impl StandaloneInformationExtension { @@ -677,8 +687,15 @@ impl StandaloneInformationExtension { region_server, procedure_manager, start_time_ms: common_time::util::current_time_millis() as u64, + flow_worker_manager: RwLock::new(None), } } + + /// Set the flow worker manager for the standalone instance. + pub async fn set_flow_worker_manager(&self, flow_worker_manager: Arc) { + let mut guard = self.flow_worker_manager.write().await; + *guard = Some(flow_worker_manager); + } } #[async_trait::async_trait] @@ -750,6 +767,18 @@ impl InformationExtension for StandaloneInformationExtension { .collect::>(); Ok(stats) } + + async fn flow_stats(&self) -> std::result::Result, Self::Error> { + Ok(Some( + self.flow_worker_manager + .read() + .await + .as_ref() + .unwrap() + .gen_state_report() + .await, + )) + } } #[cfg(test)] diff --git a/src/common/meta/src/key.rs b/src/common/meta/src/key.rs index 90b96f32dc9e..b6aa57d497cc 100644 --- a/src/common/meta/src/key.rs +++ b/src/common/meta/src/key.rs @@ -137,6 +137,7 @@ use self::schema_name::{SchemaManager, SchemaNameKey, SchemaNameValue}; use self::table_route::{TableRouteManager, TableRouteValue}; use self::tombstone::TombstoneManager; use crate::error::{self, Result, SerdeJsonSnafu}; +use crate::key::flow::flow_state::FlowStateValue; use crate::key::node_address::NodeAddressValue; use crate::key::table_route::TableRouteKey; use crate::key::txn_helper::TxnOpGetResponseSet; @@ -1262,7 +1263,8 @@ impl_metadata_value! { FlowRouteValue, TableFlowValue, NodeAddressValue, - SchemaNameValue + SchemaNameValue, + FlowStateValue } impl_optional_metadata_value! { diff --git a/src/common/meta/src/key/flow.rs b/src/common/meta/src/key/flow.rs index 9715aab1fde1..9023ca2ef83d 100644 --- a/src/common/meta/src/key/flow.rs +++ b/src/common/meta/src/key/flow.rs @@ -15,6 +15,7 @@ pub mod flow_info; pub(crate) mod flow_name; pub(crate) mod flow_route; +pub mod flow_state; pub(crate) mod flownode_flow; pub(crate) mod table_flow; @@ -35,6 +36,7 @@ use crate::ensure_values; use crate::error::{self, Result}; use crate::key::flow::flow_info::FlowInfoManager; use crate::key::flow::flow_name::FlowNameManager; +use crate::key::flow::flow_state::FlowStateManager; use crate::key::flow::flownode_flow::FlownodeFlowManager; pub use crate::key::flow::table_flow::{TableFlowManager, TableFlowManagerRef}; use crate::key::txn_helper::TxnOpGetResponseSet; @@ -102,6 +104,8 @@ pub struct FlowMetadataManager { flownode_flow_manager: FlownodeFlowManager, table_flow_manager: TableFlowManager, flow_name_manager: FlowNameManager, + /// only metasrv have access to itself's memory backend, so for other case it should be None + flow_state_manager: Option, kv_backend: KvBackendRef, } @@ -114,6 +118,7 @@ impl FlowMetadataManager { flow_name_manager: FlowNameManager::new(kv_backend.clone()), flownode_flow_manager: FlownodeFlowManager::new(kv_backend.clone()), table_flow_manager: TableFlowManager::new(kv_backend.clone()), + flow_state_manager: None, kv_backend, } } @@ -123,6 +128,10 @@ impl FlowMetadataManager { &self.flow_name_manager } + pub fn flow_state_manager(&self) -> Option<&FlowStateManager> { + self.flow_state_manager.as_ref() + } + /// Returns the [`FlowInfoManager`]. pub fn flow_info_manager(&self) -> &FlowInfoManager { &self.flow_info_manager diff --git a/src/common/meta/src/key/flow/flow_state.rs b/src/common/meta/src/key/flow/flow_state.rs new file mode 100644 index 000000000000..eeb4b06f0132 --- /dev/null +++ b/src/common/meta/src/key/flow/flow_state.rs @@ -0,0 +1,162 @@ +// 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::BTreeMap; +use std::sync::Arc; + +use serde::{Deserialize, Serialize}; + +use crate::error::{self, Result}; +use crate::key::flow::FlowScoped; +use crate::key::{FlowId, MetadataKey, MetadataValue}; +use crate::kv_backend::KvBackendRef; +use crate::rpc::store::PutRequest; + +/// The entire FlowId to Flow Size's Map is stored directly in the value part of the key. +const FLOW_STATE_KEY: &str = "state"; + +/// The key of flow state. +#[derive(Debug, Clone, Copy, PartialEq)] +struct FlowStateKeyInner; + +impl FlowStateKeyInner { + pub fn new() -> Self { + Self + } +} + +impl<'a> MetadataKey<'a, FlowStateKeyInner> for FlowStateKeyInner { + fn to_bytes(&self) -> Vec { + FLOW_STATE_KEY.as_bytes().to_vec() + } + + fn from_bytes(bytes: &'a [u8]) -> Result { + let key = std::str::from_utf8(bytes).map_err(|e| { + error::InvalidMetadataSnafu { + err_msg: format!( + "FlowInfoKeyInner '{}' is not a valid UTF8 string: {e}", + String::from_utf8_lossy(bytes) + ), + } + .build() + })?; + if key != FLOW_STATE_KEY { + return Err(error::InvalidMetadataSnafu { + err_msg: format!("Invalid FlowStateKeyInner '{key}'"), + } + .build()); + } + Ok(FlowStateKeyInner::new()) + } +} + +/// The key stores the state size of the flow. +/// +/// The layout: `__flow/state`. +pub struct FlowStateKey(FlowScoped); + +impl FlowStateKey { + /// Returns the [FlowStateKey]. + pub fn new() -> FlowStateKey { + let inner = FlowStateKeyInner::new(); + FlowStateKey(FlowScoped::new(inner)) + } +} + +impl Default for FlowStateKey { + fn default() -> Self { + Self::new() + } +} + +impl<'a> MetadataKey<'a, FlowStateKey> for FlowStateKey { + fn to_bytes(&self) -> Vec { + self.0.to_bytes() + } + + fn from_bytes(bytes: &'a [u8]) -> Result { + Ok(FlowStateKey(FlowScoped::::from_bytes( + bytes, + )?)) + } +} + +/// The value of flow state size +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] +pub struct FlowStateValue { + /// For each key, the bytes of the state in memory + pub state_size: BTreeMap, +} + +impl FlowStateValue { + pub fn new(state_size: BTreeMap) -> Self { + Self { state_size } + } +} + +pub type FlowStateManagerRef = Arc; + +/// The manager of [FlowStateKey]. Since state size changes frequently, we store it in memory. +/// +/// This is only used in distributed mode. When meta-srv use heartbeat to update the flow stat report +/// and frontned use get to get the latest flow stat report. +pub struct FlowStateManager { + in_memory: KvBackendRef, +} + +impl FlowStateManager { + pub fn new(in_memory: KvBackendRef) -> Self { + Self { in_memory } + } + + pub async fn get(&self) -> Result> { + let key = FlowStateKey::new().to_bytes(); + self.in_memory + .get(&key) + .await? + .map(|x| FlowStateValue::try_from_raw_value(&x.value)) + .transpose() + } + + pub async fn put(&self, value: FlowStateValue) -> Result<()> { + let key = FlowStateKey::new().to_bytes(); + let value = value.try_as_raw_value()?; + let req = PutRequest::new().with_key(key).with_value(value); + self.in_memory.put(req).await?; + Ok(()) + } +} + +/// Flow's state report, send regularly through heartbeat message +#[derive(Debug, Clone)] +pub struct FlowStat { + /// For each key, the bytes of the state in memory + pub state_size: BTreeMap, +} + +impl From for FlowStat { + fn from(value: FlowStateValue) -> Self { + Self { + state_size: value.state_size, + } + } +} + +impl From for FlowStateValue { + fn from(value: FlowStat) -> Self { + Self { + state_size: value.state_size, + } + } +} diff --git a/src/common/meta/src/kv_backend.rs b/src/common/meta/src/kv_backend.rs index ba9db2ec2a44..d4b2cf2ef381 100644 --- a/src/common/meta/src/kv_backend.rs +++ b/src/common/meta/src/kv_backend.rs @@ -36,7 +36,7 @@ pub mod postgres; pub mod test; pub mod txn; -pub type KvBackendRef = Arc + Send + Sync>; +pub type KvBackendRef = Arc + Send + Sync>; #[async_trait] pub trait KvBackend: TxnService @@ -161,6 +161,9 @@ where Self::Error: ErrorExt, { fn reset(&self); + + /// Upcast as `KvBackendRef`. Since https://github.com/rust-lang/rust/issues/65991 is not yet stable. + fn as_kv_backend_ref(self: Arc) -> KvBackendRef; } -pub type ResettableKvBackendRef = Arc + Send + Sync>; +pub type ResettableKvBackendRef = Arc + Send + Sync>; diff --git a/src/common/meta/src/kv_backend/memory.rs b/src/common/meta/src/kv_backend/memory.rs index 256e31f93ed3..9475a30001ce 100644 --- a/src/common/meta/src/kv_backend/memory.rs +++ b/src/common/meta/src/kv_backend/memory.rs @@ -16,13 +16,13 @@ use std::any::Any; use std::collections::BTreeMap; use std::fmt::{Display, Formatter}; use std::marker::PhantomData; -use std::sync::RwLock; +use std::sync::{Arc, RwLock}; use async_trait::async_trait; use common_error::ext::ErrorExt; use serde::Serializer; -use super::ResettableKvBackend; +use super::{KvBackendRef, ResettableKvBackend}; use crate::kv_backend::txn::{Txn, TxnOp, TxnOpResponse, TxnRequest, TxnResponse}; use crate::kv_backend::{KvBackend, TxnService}; use crate::metrics::METRIC_META_TXN_REQUEST; @@ -311,6 +311,10 @@ impl ResettableKvBackend for MemoryKvBacken fn reset(&self) { self.clear(); } + + fn as_kv_backend_ref(self: Arc) -> KvBackendRef { + self + } } #[cfg(test)] diff --git a/src/flow/Cargo.toml b/src/flow/Cargo.toml index ffba0618daaf..08867d342a74 100644 --- a/src/flow/Cargo.toml +++ b/src/flow/Cargo.toml @@ -40,6 +40,8 @@ datatypes.workspace = true enum-as-inner = "0.6.0" enum_dispatch = "0.3" futures = "0.3" +get-size-derive2 = "0.1.2" +get-size2 = "0.1.2" greptime-proto.workspace = true # This fork of hydroflow is simply for keeping our dependency in our org, and pin the version # otherwise it is the same with upstream repo diff --git a/src/flow/src/adapter.rs b/src/flow/src/adapter.rs index 7d9ae5e422d2..586eaa8e586a 100644 --- a/src/flow/src/adapter.rs +++ b/src/flow/src/adapter.rs @@ -60,6 +60,7 @@ use crate::repr::{self, DiffRow, Row, BATCH_SIZE}; mod flownode_impl; mod parse_expr; +mod stat; #[cfg(test)] mod tests; mod util; @@ -69,6 +70,7 @@ pub(crate) mod node_context; mod table_source; use crate::error::Error; +use crate::utils::StateReportHandler; use crate::FrontendInvoker; // `GREPTIME_TIMESTAMP` is not used to distinguish when table is created automatically by flow @@ -137,6 +139,8 @@ pub struct FlowWorkerManager { /// /// So that a series of event like `inserts -> flush` can be handled correctly flush_lock: RwLock<()>, + /// receive a oneshot sender to send state size report + state_report_handler: RwLock>, } /// Building FlownodeManager @@ -170,9 +174,15 @@ impl FlowWorkerManager { tick_manager, node_id, flush_lock: RwLock::new(()), + state_report_handler: RwLock::new(None), } } + pub async fn with_state_report_handler(self, handler: StateReportHandler) -> Self { + *self.state_report_handler.write().await = Some(handler); + self + } + /// Create a flownode manager with one worker pub fn new_with_worker<'s>( node_id: Option, @@ -500,6 +510,27 @@ impl FlowWorkerManager { /// Flow Runtime related methods impl FlowWorkerManager { + /// Start state report handler, which will receive a sender from HeartbeatTask to send state size report back + /// + /// if heartbeat task is shutdown, this future will exit too + async fn start_state_report_handler(self: Arc) -> Option> { + let state_report_handler = self.state_report_handler.write().await.take(); + if let Some(mut handler) = state_report_handler { + let zelf = self.clone(); + let handler = common_runtime::spawn_global(async move { + while let Some(ret_handler) = handler.recv().await { + let state_report = zelf.gen_state_report().await; + ret_handler.send(state_report).unwrap_or_else(|err| { + common_telemetry::error!(err; "Send state size report error"); + }); + } + }); + Some(handler) + } else { + None + } + } + /// run in common_runtime background runtime pub fn run_background( self: Arc, @@ -507,6 +538,7 @@ impl FlowWorkerManager { ) -> JoinHandle<()> { info!("Starting flownode manager's background task"); common_runtime::spawn_global(async move { + let _state_report_handler = self.clone().start_state_report_handler().await; self.run(shutdown).await; }) } diff --git a/src/flow/src/adapter/stat.rs b/src/flow/src/adapter/stat.rs new file mode 100644 index 000000000000..c719e35f3ca9 --- /dev/null +++ b/src/flow/src/adapter/stat.rs @@ -0,0 +1,40 @@ +// 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::BTreeMap; + +use common_meta::key::flow::flow_state::FlowStat; + +use crate::FlowWorkerManager; + +impl FlowWorkerManager { + pub async fn gen_state_report(&self) -> FlowStat { + let mut full_report = BTreeMap::new(); + for worker in self.worker_handles.iter() { + let worker = worker.lock().await; + match worker.get_state_size().await { + Ok(state_size) => { + full_report.extend(state_size.into_iter().map(|(k, v)| (k as u32, v))) + } + Err(err) => { + common_telemetry::error!(err; "Get flow stat size error"); + } + } + } + + FlowStat { + state_size: full_report, + } + } +} diff --git a/src/flow/src/adapter/worker.rs b/src/flow/src/adapter/worker.rs index 978d3c608cec..4a6b0ba963d9 100644 --- a/src/flow/src/adapter/worker.rs +++ b/src/flow/src/adapter/worker.rs @@ -197,6 +197,21 @@ impl WorkerHandle { .fail() } } + + pub async fn get_state_size(&self) -> Result, Error> { + let ret = self + .itc_client + .call_with_resp(Request::QueryStateSize) + .await?; + ret.into_query_state_size().map_err(|ret| { + InternalSnafu { + reason: format!( + "Flow Node/Worker itc failed, expect Response::QueryStateSize, found {ret:?}" + ), + } + .build() + }) + } } impl Drop for WorkerHandle { @@ -361,6 +376,13 @@ impl<'s> Worker<'s> { Some(Response::ContainTask { result: ret }) } Request::Shutdown => return Err(()), + Request::QueryStateSize => { + let mut ret = BTreeMap::new(); + for (flow_id, task_state) in self.task_states.iter() { + ret.insert(*flow_id, task_state.state.get_state_size()); + } + Some(Response::QueryStateSize { result: ret }) + } }; Ok(ret) } @@ -391,6 +413,7 @@ pub enum Request { flow_id: FlowId, }, Shutdown, + QueryStateSize, } #[derive(Debug, EnumAsInner)] @@ -406,6 +429,10 @@ enum Response { result: bool, }, RunAvail, + QueryStateSize { + /// each flow tasks' state size + result: BTreeMap, + }, } fn create_inter_thread_call() -> (InterThreadCallClient, InterThreadCallServer) { @@ -423,10 +450,12 @@ struct InterThreadCallClient { } impl InterThreadCallClient { + /// call without response fn call_no_resp(&self, req: Request) -> Result<(), Error> { self.arg_sender.send((req, None)).map_err(from_send_error) } + /// call with response async fn call_with_resp(&self, req: Request) -> Result { let (tx, rx) = oneshot::channel(); self.arg_sender @@ -527,6 +556,7 @@ mod test { ); tx.send(Batch::empty()).unwrap(); handle.run_available(0, true).await.unwrap(); + assert_eq!(handle.get_state_size().await.unwrap().len(), 1); assert_eq!(sink_rx.recv().await.unwrap(), Batch::empty()); drop(handle); worker_thread_handle.join().unwrap(); diff --git a/src/flow/src/compute/state.rs b/src/flow/src/compute/state.rs index d34b4a311d15..ee2c7628a28c 100644 --- a/src/flow/src/compute/state.rs +++ b/src/flow/src/compute/state.rs @@ -16,6 +16,7 @@ use std::cell::RefCell; use std::collections::{BTreeMap, VecDeque}; use std::rc::Rc; +use get_size2::GetSize; use hydroflow::scheduled::graph::Hydroflow; use hydroflow::scheduled::SubgraphId; @@ -109,6 +110,10 @@ impl DataflowState { pub fn expire_after(&self) -> Option { self.expire_after } + + pub fn get_state_size(&self) -> usize { + self.arrange_used.iter().map(|x| x.read().get_size()).sum() + } } #[derive(Debug, Clone)] diff --git a/src/flow/src/heartbeat.rs b/src/flow/src/heartbeat.rs index 96635e350dde..69159d1d2a01 100644 --- a/src/flow/src/heartbeat.rs +++ b/src/flow/src/heartbeat.rs @@ -24,6 +24,7 @@ use common_meta::heartbeat::handler::{ }; use common_meta::heartbeat::mailbox::{HeartbeatMailbox, MailboxRef, OutgoingMessage}; use common_meta::heartbeat::utils::outgoing_message_to_mailbox_message; +use common_meta::key::flow::flow_state::FlowStat; use common_telemetry::{debug, error, info, warn}; use greptime_proto::v1::meta::NodeInfo; use meta_client::client::{HeartbeatSender, HeartbeatStream, MetaClient}; @@ -34,8 +35,27 @@ use tokio::sync::mpsc; use tokio::time::Duration; use crate::error::ExternalSnafu; +use crate::utils::SizeReportSender; use crate::{Error, FlownodeOptions}; +async fn query_flow_state( + query_stat_size: &Option, + timeout: Duration, +) -> Option { + if let Some(report_requester) = query_stat_size.as_ref() { + let ret = report_requester.query(timeout).await; + match ret { + Ok(latest) => Some(latest), + Err(err) => { + error!(err; "Failed to get query stat size"); + None + } + } + } else { + None + } +} + /// The flownode heartbeat task which sending `[HeartbeatRequest]` to Metasrv periodically in background. #[derive(Clone)] pub struct HeartbeatTask { @@ -47,9 +67,14 @@ pub struct HeartbeatTask { resp_handler_executor: HeartbeatResponseHandlerExecutorRef, start_time_ms: u64, running: Arc, + query_stat_size: Option, } impl HeartbeatTask { + pub fn with_query_stat_size(mut self, query_stat_size: SizeReportSender) -> Self { + self.query_stat_size = Some(query_stat_size); + self + } pub fn new( opts: &FlownodeOptions, meta_client: Arc, @@ -65,6 +90,7 @@ impl HeartbeatTask { resp_handler_executor, start_time_ms: common_time::util::current_time_millis() as u64, running: Arc::new(AtomicBool::new(false)), + query_stat_size: None, } } @@ -112,6 +138,7 @@ impl HeartbeatTask { message: Option, peer: Option, start_time_ms: u64, + latest_report: &Option, ) -> Option { let mailbox_message = match message.map(outgoing_message_to_mailbox_message) { Some(Ok(message)) => Some(message), @@ -121,11 +148,22 @@ impl HeartbeatTask { } None => None, }; + let flow_stat = latest_report + .as_ref() + .map(|report| { + report + .state_size + .iter() + .map(|(k, v)| (*k, *v as u64)) + .collect() + }) + .map(|f| api::v1::meta::FlowStat { flow_stat_size: f }); Some(HeartbeatRequest { mailbox_message, peer, info: Self::build_node_info(start_time_ms), + flow_stat, ..Default::default() }) } @@ -151,24 +189,27 @@ impl HeartbeatTask { addr: self.peer_addr.clone(), }); + let query_stat_size = self.query_stat_size.clone(); + common_runtime::spawn_hb(async move { // note that using interval will cause it to first immediately send // a heartbeat let mut interval = tokio::time::interval(report_interval); interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + let mut latest_report = None; loop { let req = tokio::select! { message = outgoing_rx.recv() => { if let Some(message) = message { - Self::create_heartbeat_request(Some(message), self_peer.clone(), start_time_ms) + Self::create_heartbeat_request(Some(message), self_peer.clone(), start_time_ms, &latest_report) } else { // Receives None that means Sender was dropped, we need to break the current loop break } } _ = interval.tick() => { - Self::create_heartbeat_request(None, self_peer.clone(), start_time_ms) + Self::create_heartbeat_request(None, self_peer.clone(), start_time_ms, &latest_report) } }; @@ -180,6 +221,10 @@ impl HeartbeatTask { debug!("Send a heartbeat request to metasrv, content: {:?}", req); } } + // after sending heartbeat, try to get the latest report + // TODO(discord9): consider a better place to update the size report + // set the timeout to half of the report interval so that it wouldn't delay heartbeat if something went horribly wrong + latest_report = query_flow_state(&query_stat_size, report_interval / 2).await; } }); } diff --git a/src/flow/src/repr.rs b/src/flow/src/repr.rs index 1c257c8f3c6c..50f2a78ef8f9 100644 --- a/src/flow/src/repr.rs +++ b/src/flow/src/repr.rs @@ -22,12 +22,14 @@ use api::v1::Row as ProtoRow; use datatypes::data_type::ConcreteDataType; use datatypes::types::cast; use datatypes::value::Value; +use get_size2::GetSize; use itertools::Itertools; pub(crate) use relation::{ColumnType, Key, RelationDesc, RelationType}; use serde::{Deserialize, Serialize}; use snafu::ResultExt; use crate::expr::error::{CastValueSnafu, EvalError, InvalidArgumentSnafu}; +use crate::utils::get_value_heap_size; /// System-wide Record count difference type. Useful for capture data change /// @@ -105,6 +107,12 @@ pub struct Row { pub inner: Vec, } +impl GetSize for Row { + fn get_heap_size(&self) -> usize { + self.inner.iter().map(get_value_heap_size).sum() + } +} + impl Row { /// Create an empty row pub fn empty() -> Self { diff --git a/src/flow/src/server.rs b/src/flow/src/server.rs index 87b6bbdc09ed..1259c1175510 100644 --- a/src/flow/src/server.rs +++ b/src/flow/src/server.rs @@ -55,6 +55,7 @@ use crate::error::{ }; use crate::heartbeat::HeartbeatTask; use crate::transform::register_function_to_query_engine; +use crate::utils::{SizeReportSender, StateReportHandler}; use crate::{Error, FlowWorkerManager, FlownodeOptions}; pub const FLOW_NODE_SERVER_NAME: &str = "FLOW_NODE_SERVER"; @@ -236,6 +237,8 @@ pub struct FlownodeBuilder { catalog_manager: CatalogManagerRef, flow_metadata_manager: FlowMetadataManagerRef, heartbeat_task: Option, + /// receive a oneshot sender to send state size report + state_report_handler: Option, } impl FlownodeBuilder { @@ -254,17 +257,20 @@ impl FlownodeBuilder { catalog_manager, flow_metadata_manager, heartbeat_task: None, + state_report_handler: None, } } pub fn with_heartbeat_task(self, heartbeat_task: HeartbeatTask) -> Self { + let (sender, receiver) = SizeReportSender::new(); Self { - heartbeat_task: Some(heartbeat_task), + heartbeat_task: Some(heartbeat_task.with_query_stat_size(sender)), + state_report_handler: Some(receiver), ..self } } - pub async fn build(self) -> Result { + pub async fn build(mut self) -> Result { // TODO(discord9): does this query engine need those? let query_engine_factory = QueryEngineFactory::new_with_plugins( // query engine in flownode is only used for translate plan with resolved table source. @@ -383,7 +389,7 @@ impl FlownodeBuilder { /// build [`FlowWorkerManager`], note this doesn't take ownership of `self`, /// nor does it actually start running the worker. async fn build_manager( - &self, + &mut self, query_engine: Arc, ) -> Result { let table_meta = self.table_meta.clone(); @@ -402,12 +408,15 @@ impl FlownodeBuilder { info!("Flow Worker started in new thread"); worker.run(); }); - let man = rx.await.map_err(|_e| { + let mut man = rx.await.map_err(|_e| { UnexpectedSnafu { reason: "sender is dropped, failed to create flow node manager", } .build() })?; + if let Some(handler) = self.state_report_handler.take() { + man = man.with_state_report_handler(handler).await; + } info!("Flow Node Manager started"); Ok(man) } diff --git a/src/flow/src/utils.rs b/src/flow/src/utils.rs index 1cd5b3ba5c1c..5e01d0bfa423 100644 --- a/src/flow/src/utils.rs +++ b/src/flow/src/utils.rs @@ -18,16 +18,73 @@ use std::collections::{BTreeMap, BTreeSet}; use std::ops::Bound; use std::sync::Arc; +use common_meta::key::flow::flow_state::FlowStat; use common_telemetry::trace; +use datatypes::value::Value; +use get_size2::GetSize; use smallvec::{smallvec, SmallVec}; -use tokio::sync::RwLock; +use tokio::sync::{mpsc, oneshot, RwLock}; +use tokio::time::Instant; +use crate::error::InternalSnafu; use crate::expr::{EvalError, ScalarExpr}; use crate::repr::{value_to_internal_ts, DiffRow, Duration, KeyValDiffRow, Row, Timestamp}; /// A batch of updates, arranged by key pub type Batch = BTreeMap>; +/// Get a estimate of heap size of a value +pub fn get_value_heap_size(v: &Value) -> usize { + match v { + Value::Binary(bin) => bin.len(), + Value::String(s) => s.len(), + Value::List(list) => list.items().iter().map(get_value_heap_size).sum(), + _ => 0, + } +} + +#[derive(Clone)] +pub struct SizeReportSender { + inner: mpsc::Sender>, +} + +impl SizeReportSender { + pub fn new() -> (Self, StateReportHandler) { + let (tx, rx) = mpsc::channel(1); + let zelf = Self { inner: tx }; + (zelf, rx) + } + + /// Query the size report, will timeout after one second if no response + pub async fn query(&self, timeout: std::time::Duration) -> crate::Result { + let (tx, rx) = oneshot::channel(); + self.inner.send(tx).await.map_err(|_| { + InternalSnafu { + reason: "failed to send size report request due to receiver dropped", + } + .build() + })?; + let timeout = tokio::time::timeout(timeout, rx); + timeout + .await + .map_err(|_elapsed| { + InternalSnafu { + reason: "failed to receive size report after one second timeout", + } + .build() + })? + .map_err(|_| { + InternalSnafu { + reason: "failed to receive size report due to sender dropped", + } + .build() + }) + } +} + +/// Handle the size report request, and send the report back +pub type StateReportHandler = mpsc::Receiver>; + /// A spine of batches, arranged by timestamp /// TODO(discord9): consider internally index by key, value, and timestamp for faster lookup pub type Spine = BTreeMap; @@ -49,6 +106,24 @@ pub struct KeyExpiryManager { event_timestamp_from_row: Option, } +impl GetSize for KeyExpiryManager { + fn get_heap_size(&self) -> usize { + let row_size = if let Some(row_size) = &self + .event_ts_to_key + .first_key_value() + .map(|(_, v)| v.first().get_heap_size()) + { + *row_size + } else { + 0 + }; + self.event_ts_to_key + .values() + .map(|v| v.len() * row_size + std::mem::size_of::()) + .sum::() + } +} + impl KeyExpiryManager { pub fn new( key_expiration_duration: Option, @@ -154,7 +229,7 @@ impl KeyExpiryManager { /// /// Note the two way arrow between reduce operator and arrange, it's because reduce operator need to query existing state /// and also need to update existing state. -#[derive(Debug, Clone, Default, Eq, PartialEq, Ord, PartialOrd)] +#[derive(Debug, Clone, Eq, PartialEq, Ord, PartialOrd)] pub struct Arrangement { /// A name or identifier for the arrangement which can be used for debugging or logging purposes. /// This field is not critical to the functionality but aids in monitoring and management of arrangements. @@ -196,6 +271,61 @@ pub struct Arrangement { /// The time that the last compaction happened, also known as the current time. last_compaction_time: Option, + + /// Estimated size of the arrangement in heap size. + estimated_size: usize, + last_size_update: Instant, + size_update_interval: tokio::time::Duration, +} + +impl Arrangement { + fn compute_size(&self) -> usize { + self.spine + .values() + .map(|v| { + let per_entry_size = v + .first_key_value() + .map(|(k, v)| { + k.get_heap_size() + + v.len() * v.first().map(|r| r.get_heap_size()).unwrap_or(0) + }) + .unwrap_or(0); + std::mem::size_of::() + v.len() * per_entry_size + }) + .sum::() + + self.expire_state.get_heap_size() + + self.name.get_heap_size() + } + + fn update_and_fetch_size(&mut self) -> usize { + if self.last_size_update.elapsed() > self.size_update_interval { + self.estimated_size = self.compute_size(); + self.last_size_update = Instant::now(); + } + self.estimated_size + } +} + +impl GetSize for Arrangement { + fn get_heap_size(&self) -> usize { + self.estimated_size + } +} + +impl Default for Arrangement { + fn default() -> Self { + Self { + spine: Default::default(), + full_arrangement: false, + is_written: false, + expire_state: None, + last_compaction_time: None, + name: Vec::new(), + estimated_size: 0, + last_size_update: Instant::now(), + size_update_interval: tokio::time::Duration::from_secs(3), + } + } } impl Arrangement { @@ -207,6 +337,9 @@ impl Arrangement { expire_state: None, last_compaction_time: None, name, + estimated_size: 0, + last_size_update: Instant::now(), + size_update_interval: tokio::time::Duration::from_secs(3), } } @@ -269,6 +402,7 @@ impl Arrangement { // without changing the order of updates within same tick key_updates.sort_by_key(|(_val, ts, _diff)| *ts); } + self.update_and_fetch_size(); Ok(max_expired_by) } @@ -390,6 +524,7 @@ impl Arrangement { // insert the compacted batch into spine with key being `now` self.spine.insert(now, compacting_batch); + self.update_and_fetch_size(); Ok(max_expired_by) } diff --git a/src/meta-client/src/client.rs b/src/meta-client/src/client.rs index d0008a7e81b1..ebe0e94e4861 100644 --- a/src/meta-client/src/client.rs +++ b/src/meta-client/src/client.rs @@ -25,6 +25,7 @@ use std::sync::Arc; use api::v1::meta::{ProcedureDetailResponse, Role}; use cluster::Client as ClusterClient; +pub use cluster::ClusterKvBackend; use common_error::ext::BoxedError; use common_grpc::channel_manager::{ChannelConfig, ChannelManager}; use common_meta::cluster::{ @@ -33,6 +34,8 @@ use common_meta::cluster::{ use common_meta::datanode::{DatanodeStatKey, DatanodeStatValue, RegionStat}; use common_meta::ddl::{ExecutorContext, ProcedureExecutor}; use common_meta::error::{self as meta_error, ExternalSnafu, Result as MetaResult}; +use common_meta::key::flow::flow_state::{FlowStat, FlowStateManager}; +use common_meta::kv_backend::KvBackendRef; use common_meta::range_stream::PaginationStream; use common_meta::rpc::ddl::{SubmitDdlTaskRequest, SubmitDdlTaskResponse}; use common_meta::rpc::procedure::{ @@ -54,7 +57,8 @@ use store::Client as StoreClient; pub use self::heartbeat::{HeartbeatSender, HeartbeatStream}; use crate::error::{ - ConvertMetaRequestSnafu, ConvertMetaResponseSnafu, Error, NotStartedSnafu, Result, + ConvertMetaRequestSnafu, ConvertMetaResponseSnafu, Error, GetFlowStatSnafu, NotStartedSnafu, + Result, }; pub type Id = (u64, u64); @@ -347,6 +351,15 @@ fn decode_stats(kv: KeyValue) -> MetaResult { } impl MetaClient { + pub async fn list_flow_stats(&self) -> Result> { + let cluster_backend = ClusterKvBackend::new(Arc::new(self.cluster_client()?)); + let cluster_backend = Arc::new(cluster_backend) as KvBackendRef; + let flow_state_manager = FlowStateManager::new(cluster_backend); + let res = flow_state_manager.get().await.context(GetFlowStatSnafu)?; + + Ok(res.map(|r| r.into())) + } + pub fn new(id: Id) -> Self { Self { id, diff --git a/src/meta-client/src/client/cluster.rs b/src/meta-client/src/client/cluster.rs index c7edbcc8d39e..d50ac9717cc8 100644 --- a/src/meta-client/src/client/cluster.rs +++ b/src/meta-client/src/client/cluster.rs @@ -40,8 +40,8 @@ use tonic::Status; use crate::client::ask_leader::AskLeader; use crate::client::{util, Id}; use crate::error::{ - ConvertMetaResponseSnafu, CreateChannelSnafu, Error, IllegalGrpcClientStateSnafu, Result, - RetryTimesExceededSnafu, + ConvertMetaResponseSnafu, CreateChannelSnafu, Error, IllegalGrpcClientStateSnafu, + ReadOnlyKvBackendSnafu, Result, RetryTimesExceededSnafu, }; #[derive(Clone, Debug)] @@ -308,3 +308,75 @@ impl Inner { .map(|res| (res.leader, res.followers)) } } + +/// A client for the cluster info. Read only and corresponding to +/// `in_memory` kvbackend in the meta-srv. +#[derive(Clone, Debug)] +pub struct ClusterKvBackend { + inner: Arc, +} + +impl ClusterKvBackend { + pub fn new(client: Arc) -> Self { + Self { inner: client } + } + + fn unimpl(&self) -> common_meta::error::Error { + let ret: common_meta::error::Result<()> = ReadOnlyKvBackendSnafu { + name: self.name().to_string(), + } + .fail() + .map_err(BoxedError::new) + .context(common_meta::error::ExternalSnafu); + ret.unwrap_err() + } +} + +impl TxnService for ClusterKvBackend { + type Error = common_meta::error::Error; +} + +#[async_trait::async_trait] +impl KvBackend for ClusterKvBackend { + fn name(&self) -> &str { + "ClusterKvBackend" + } + + fn as_any(&self) -> &dyn Any { + self + } + + async fn range(&self, req: RangeRequest) -> common_meta::error::Result { + self.inner + .range(req) + .await + .map_err(BoxedError::new) + .context(common_meta::error::ExternalSnafu) + } + + async fn batch_get(&self, _: BatchGetRequest) -> common_meta::error::Result { + Err(self.unimpl()) + } + + async fn put(&self, _: PutRequest) -> common_meta::error::Result { + Err(self.unimpl()) + } + + async fn batch_put(&self, _: BatchPutRequest) -> common_meta::error::Result { + Err(self.unimpl()) + } + + async fn delete_range( + &self, + _: DeleteRangeRequest, + ) -> common_meta::error::Result { + Err(self.unimpl()) + } + + async fn batch_delete( + &self, + _: BatchDeleteRequest, + ) -> common_meta::error::Result { + Err(self.unimpl()) + } +} diff --git a/src/meta-client/src/error.rs b/src/meta-client/src/error.rs index a4f8663368b4..be1cf150da0f 100644 --- a/src/meta-client/src/error.rs +++ b/src/meta-client/src/error.rs @@ -99,8 +99,22 @@ pub enum Error { source: common_meta::error::Error, }, + #[snafu(display("Failed to get flow stat"))] + GetFlowStat { + #[snafu(implicit)] + location: Location, + source: common_meta::error::Error, + }, + #[snafu(display("Retry exceeded max times({}), message: {}", times, msg))] RetryTimesExceeded { times: usize, msg: String }, + + #[snafu(display("Trying to write to a read-only kv backend: {}", name))] + ReadOnlyKvBackend { + name: String, + #[snafu(implicit)] + location: Location, + }, } #[allow(dead_code)] @@ -120,13 +134,15 @@ impl ErrorExt for Error { | Error::SendHeartbeat { .. } | Error::CreateHeartbeatStream { .. } | Error::CreateChannel { .. } - | Error::RetryTimesExceeded { .. } => StatusCode::Internal, + | Error::RetryTimesExceeded { .. } + | Error::ReadOnlyKvBackend { .. } => StatusCode::Internal, Error::MetaServer { code, .. } => *code, Error::InvalidResponseHeader { source, .. } | Error::ConvertMetaRequest { source, .. } - | Error::ConvertMetaResponse { source, .. } => source.status_code(), + | Error::ConvertMetaResponse { source, .. } + | Error::GetFlowStat { source, .. } => source.status_code(), } } } diff --git a/src/meta-srv/src/error.rs b/src/meta-srv/src/error.rs index 705f31ac49f4..1c529f06d606 100644 --- a/src/meta-srv/src/error.rs +++ b/src/meta-srv/src/error.rs @@ -716,6 +716,13 @@ pub enum Error { #[snafu(implicit)] location: Location, }, + + #[snafu(display("Flow state handler error"))] + FlowStateHandler { + #[snafu(implicit)] + location: Location, + source: common_meta::error::Error, + }, } impl Error { @@ -761,7 +768,8 @@ impl ErrorExt for Error { | Error::Join { .. } | Error::PeerUnavailable { .. } | Error::ExceededDeadline { .. } - | Error::ChooseItems { .. } => StatusCode::Internal, + | Error::ChooseItems { .. } + | Error::FlowStateHandler { .. } => StatusCode::Internal, Error::Unsupported { .. } => StatusCode::Unsupported, diff --git a/src/meta-srv/src/handler.rs b/src/meta-srv/src/handler.rs index 3b4eb9a27935..ad1492cd7cdc 100644 --- a/src/meta-srv/src/handler.rs +++ b/src/meta-srv/src/handler.rs @@ -51,6 +51,7 @@ use tokio::sync::mpsc::Sender; use tokio::sync::{oneshot, Notify, RwLock}; use crate::error::{self, DeserializeFromJsonSnafu, Result, UnexpectedInstructionReplySnafu}; +use crate::handler::flow_state_handler::FlowStateHandler; use crate::metasrv::Context; use crate::metrics::{METRIC_META_HANDLER_EXECUTE, METRIC_META_HEARTBEAT_CONNECTION_NUM}; use crate::pubsub::PublisherRef; @@ -64,6 +65,7 @@ pub mod collect_stats_handler; pub mod extract_stat_handler; pub mod failure_handler; pub mod filter_inactive_region_stats; +pub mod flow_state_handler; pub mod keep_lease_handler; pub mod mailbox_handler; pub mod on_leader_start_handler; @@ -482,6 +484,8 @@ pub struct HeartbeatHandlerGroupBuilder { /// based on the number of received heartbeats. When the number of heartbeats /// reaches this factor, a flush operation is triggered. flush_stats_factor: Option, + /// A simple handler for flow internal state report + flow_state_handler: Option, /// The plugins. plugins: Option, @@ -499,12 +503,18 @@ impl HeartbeatHandlerGroupBuilder { region_failure_handler: None, region_lease_handler: None, flush_stats_factor: None, + flow_state_handler: None, plugins: None, pushers, handlers: vec![], } } + pub fn with_flow_state_handler(mut self, handler: Option) -> Self { + self.flow_state_handler = handler; + self + } + pub fn with_region_lease_handler(mut self, handler: Option) -> Self { self.region_lease_handler = handler; self @@ -564,6 +574,10 @@ impl HeartbeatHandlerGroupBuilder { } self.add_handler_last(CollectStatsHandler::new(self.flush_stats_factor)); + if let Some(flow_state_handler) = self.flow_state_handler.take() { + self.add_handler_last(flow_state_handler); + } + self } diff --git a/src/meta-srv/src/handler/flow_state_handler.rs b/src/meta-srv/src/handler/flow_state_handler.rs new file mode 100644 index 000000000000..ab387eb29185 --- /dev/null +++ b/src/meta-srv/src/handler/flow_state_handler.rs @@ -0,0 +1,58 @@ +// 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 api::v1::meta::{FlowStat, HeartbeatRequest, Role}; +use common_meta::key::flow::flow_state::{FlowStateManager, FlowStateValue}; +use snafu::ResultExt; + +use crate::error::{FlowStateHandlerSnafu, Result}; +use crate::handler::{HandleControl, HeartbeatAccumulator, HeartbeatHandler}; +use crate::metasrv::Context; + +pub struct FlowStateHandler { + flow_state_manager: FlowStateManager, +} + +impl FlowStateHandler { + pub fn new(flow_state_manager: FlowStateManager) -> Self { + Self { flow_state_manager } + } +} + +#[async_trait::async_trait] +impl HeartbeatHandler for FlowStateHandler { + fn is_acceptable(&self, role: Role) -> bool { + role == Role::Flownode + } + + async fn handle( + &self, + req: &HeartbeatRequest, + _ctx: &mut Context, + _acc: &mut HeartbeatAccumulator, + ) -> Result { + if let Some(FlowStat { flow_stat_size }) = &req.flow_stat { + let state_size = flow_stat_size + .iter() + .map(|(k, v)| (*k, *v as usize)) + .collect(); + let value = FlowStateValue::new(state_size); + self.flow_state_manager + .put(value) + .await + .context(FlowStateHandlerSnafu)?; + } + Ok(HandleControl::Continue) + } +} diff --git a/src/meta-srv/src/metasrv/builder.rs b/src/meta-srv/src/metasrv/builder.rs index 05344b482b06..0afaf004933d 100644 --- a/src/meta-srv/src/metasrv/builder.rs +++ b/src/meta-srv/src/metasrv/builder.rs @@ -26,6 +26,7 @@ use common_meta::ddl::{ }; use common_meta::ddl_manager::DdlManager; use common_meta::distributed_time_constants; +use common_meta::key::flow::flow_state::FlowStateManager; use common_meta::key::flow::FlowMetadataManager; use common_meta::key::maintenance::MaintenanceModeManager; use common_meta::key::TableMetadataManager; @@ -47,6 +48,7 @@ use crate::error::{self, Result}; use crate::flow_meta_alloc::FlowPeerAllocator; use crate::greptimedb_telemetry::get_greptimedb_telemetry_task; use crate::handler::failure_handler::RegionFailureHandler; +use crate::handler::flow_state_handler::FlowStateHandler; use crate::handler::region_lease_handler::RegionLeaseHandler; use crate::handler::{HeartbeatHandlerGroupBuilder, HeartbeatMailbox, Pushers}; use crate::lease::MetaPeerLookupService; @@ -228,6 +230,7 @@ impl MetasrvBuilder { peer_allocator, )) }); + let flow_metadata_allocator = { // for now flownode just use round-robin selector let flow_selector = RoundRobinSelector::new(SelectTarget::Flownode); @@ -248,6 +251,9 @@ impl MetasrvBuilder { peer_allocator, )) }; + let flow_state_handler = + FlowStateHandler::new(FlowStateManager::new(in_memory.clone().as_kv_backend_ref())); + let memory_region_keeper = Arc::new(MemoryRegionKeeper::default()); let node_manager = node_manager.unwrap_or_else(|| { let datanode_client_channel_config = ChannelConfig::new() @@ -350,6 +356,7 @@ impl MetasrvBuilder { .with_region_failure_handler(region_failover_handler) .with_region_lease_handler(Some(region_lease_handler)) .with_flush_stats_factor(Some(options.flush_stats_factor)) + .with_flow_state_handler(Some(flow_state_handler)) .add_default_handlers() } }; diff --git a/src/meta-srv/src/service/store/cached_kv.rs b/src/meta-srv/src/service/store/cached_kv.rs index d4b6f84f5802..0f90ecddea33 100644 --- a/src/meta-srv/src/service/store/cached_kv.rs +++ b/src/meta-srv/src/service/store/cached_kv.rs @@ -386,6 +386,10 @@ impl ResettableKvBackend for LeaderCachedKvBackend { fn reset(&self) { self.cache.reset() } + + fn as_kv_backend_ref(self: Arc) -> KvBackendRef { + self + } } #[cfg(test)] diff --git a/tests/cases/standalone/common/flow/flow_basic.result b/tests/cases/standalone/common/flow/flow_basic.result index 8ee6a90c83bf..fa360a6de684 100644 --- a/tests/cases/standalone/common/flow/flow_basic.result +++ b/tests/cases/standalone/common/flow/flow_basic.result @@ -1045,6 +1045,18 @@ FROM | svc1 | 200 | 2024-10-18T19:01:31 | +--------------+-----+---------------------+ +-- Test if FLOWS table works, but don't care about the result since it vary from runs +SELECT + count(CASE WHEN state_size > 0 THEN 1 ELSE 0 END) as active_flows, +FROM + INFORMATION_SCHEMA.FLOWS; + ++--------------+ +| active_flows | ++--------------+ +| 1 | ++--------------+ + DROP FLOW requests_long_term; Affected Rows: 0 diff --git a/tests/cases/standalone/common/flow/flow_basic.sql b/tests/cases/standalone/common/flow/flow_basic.sql index 43a42de4dd5f..8946c014be36 100644 --- a/tests/cases/standalone/common/flow/flow_basic.sql +++ b/tests/cases/standalone/common/flow/flow_basic.sql @@ -569,6 +569,12 @@ SELECT FROM requests_without_ip; +-- Test if FLOWS table works, but don't care about the result since it vary from runs +SELECT + count(CASE WHEN state_size > 0 THEN 1 ELSE 0 END) as active_flows, +FROM + INFORMATION_SCHEMA.FLOWS; + DROP FLOW requests_long_term; DROP TABLE requests_without_ip; diff --git a/tests/cases/standalone/common/system/information_schema.result b/tests/cases/standalone/common/system/information_schema.result index b1c8c9329514..a0dce0152296 100644 --- a/tests/cases/standalone/common/system/information_schema.result +++ b/tests/cases/standalone/common/system/information_schema.result @@ -187,16 +187,17 @@ select * from information_schema.columns order by table_schema, table_name, colu | greptime | information_schema | files | update_count | 13 | | | 19 | 0 | | | | | | select,insert | | Int64 | bigint | FIELD | | No | bigint | | | | greptime | information_schema | files | update_time | 34 | | | | | 3 | | | | | select,insert | | DateTime | datetime | FIELD | | No | datetime | | | | greptime | information_schema | files | version | 25 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | -| greptime | information_schema | flows | comment | 5 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | -| greptime | information_schema | flows | expire_after | 6 | | | 19 | 0 | | | | | | select,insert | | Int64 | bigint | FIELD | | Yes | bigint | | | -| greptime | information_schema | flows | flow_definition | 4 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | +| greptime | information_schema | flows | comment | 6 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | +| greptime | information_schema | flows | expire_after | 7 | | | 19 | 0 | | | | | | select,insert | | Int64 | bigint | FIELD | | Yes | bigint | | | +| greptime | information_schema | flows | flow_definition | 5 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | | greptime | information_schema | flows | flow_id | 2 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | | | greptime | information_schema | flows | flow_name | 1 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | -| greptime | information_schema | flows | flownode_ids | 9 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | -| greptime | information_schema | flows | options | 10 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | -| greptime | information_schema | flows | sink_table_name | 8 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | -| greptime | information_schema | flows | source_table_ids | 7 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | -| greptime | information_schema | flows | table_catalog | 3 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | +| greptime | information_schema | flows | flownode_ids | 10 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | +| greptime | information_schema | flows | options | 11 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | +| greptime | information_schema | flows | sink_table_name | 9 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | +| greptime | information_schema | flows | source_table_ids | 8 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | +| greptime | information_schema | flows | state_size | 3 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | | +| greptime | information_schema | flows | table_catalog | 4 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | | greptime | information_schema | global_status | variable_name | 1 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | | greptime | information_schema | global_status | variable_value | 2 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | | greptime | information_schema | key_column_usage | column_name | 8 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | From 7ea8a44d3a61427f61529f76a0f7966abe928643 Mon Sep 17 00:00:00 2001 From: dennis zhuang Date: Thu, 19 Dec 2024 16:28:20 +0800 Subject: [PATCH 02/12] chore: update PR template (#5199) --- .github/pull_request_template.md | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index c50137f87681..5b0fe1bcfe2d 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -4,7 +4,8 @@ I hereby agree to the terms of the [GreptimeDB CLA](https://github.com/GreptimeT ## What's changed and what's your intention? -__!!! DO NOT LEAVE THIS BLOCK EMPTY !!!__ + -## Checklist +## PR Checklist +Please convert it to a draft if some of the following conditions are not met. - [ ] I have written the necessary rustdoc comments. - [ ] I have added the necessary unit tests and integration tests. - [ ] This PR requires documentation updates. +- [ ] API changes are backward compatible. +- [ ] Schema or data changes are backward compatible. From a4d61bcaf188e88ebcaface8ee899ea2e6eb99bb Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Thu, 19 Dec 2024 17:16:56 +0800 Subject: [PATCH 03/12] fix(flow): batch builder with type (#5195) * fix: typed builder * chore: clippy * chore: rename * fix: unit tests * refactor: per review --- src/flow/src/adapter.rs | 22 ++- src/flow/src/adapter/flownode_impl.rs | 13 +- src/flow/src/adapter/node_context.rs | 23 ++- src/flow/src/adapter/util.rs | 15 ++ src/flow/src/compute/render.rs | 23 ++- src/flow/src/compute/render/map.rs | 3 +- src/flow/src/compute/render/reduce.rs | 35 ++-- src/flow/src/expr.rs | 37 +++- src/flow/src/expr/linear.rs | 70 +++++-- .../standalone/common/flow/flow_basic.result | 178 +++++++++++++++++- .../standalone/common/flow/flow_basic.sql | 79 +++++++- 11 files changed, 434 insertions(+), 64 deletions(-) diff --git a/src/flow/src/adapter.rs b/src/flow/src/adapter.rs index 586eaa8e586a..6d70377cf2aa 100644 --- a/src/flow/src/adapter.rs +++ b/src/flow/src/adapter.rs @@ -565,6 +565,8 @@ impl FlowWorkerManager { let default_interval = Duration::from_secs(1); let mut avg_spd = 0; // rows/sec let mut since_last_run = tokio::time::Instant::now(); + let run_per_trace = 10; + let mut run_cnt = 0; loop { // TODO(discord9): only run when new inputs arrive or scheduled to let row_cnt = self.run_available(true).await.unwrap_or_else(|err| { @@ -607,10 +609,19 @@ impl FlowWorkerManager { } else { (9 * avg_spd + cur_spd) / 10 }; - trace!("avg_spd={} r/s, cur_spd={} r/s", avg_spd, cur_spd); let new_wait = BATCH_SIZE * 1000 / avg_spd.max(1); //in ms let new_wait = Duration::from_millis(new_wait as u64).min(default_interval); - trace!("Wait for {} ms, row_cnt={}", new_wait.as_millis(), row_cnt); + + // print trace every `run_per_trace` times so that we can see if there is something wrong + // but also not get flooded with trace + if run_cnt >= run_per_trace { + trace!("avg_spd={} r/s, cur_spd={} r/s", avg_spd, cur_spd); + trace!("Wait for {} ms, row_cnt={}", new_wait.as_millis(), row_cnt); + run_cnt = 0; + } else { + run_cnt += 1; + } + METRIC_FLOW_RUN_INTERVAL_MS.set(new_wait.as_millis() as i64); since_last_run = tokio::time::Instant::now(); tokio::time::sleep(new_wait).await; @@ -670,13 +681,18 @@ impl FlowWorkerManager { &self, region_id: RegionId, rows: Vec, + batch_datatypes: &[ConcreteDataType], ) -> Result<(), Error> { let rows_len = rows.len(); let table_id = region_id.table_id(); let _timer = METRIC_FLOW_INSERT_ELAPSED .with_label_values(&[table_id.to_string().as_str()]) .start_timer(); - self.node_context.read().await.send(table_id, rows).await?; + self.node_context + .read() + .await + .send(table_id, rows, batch_datatypes) + .await?; trace!( "Handling write request for table_id={} with {} rows", table_id, diff --git a/src/flow/src/adapter/flownode_impl.rs b/src/flow/src/adapter/flownode_impl.rs index 3841d08914c5..1fa11b4d83a2 100644 --- a/src/flow/src/adapter/flownode_impl.rs +++ b/src/flow/src/adapter/flownode_impl.rs @@ -28,6 +28,7 @@ use itertools::Itertools; use snafu::{OptionExt, ResultExt}; use store_api::storage::RegionId; +use super::util::from_proto_to_data_type; use crate::adapter::{CreateFlowArgs, FlowWorkerManager}; use crate::error::InternalSnafu; use crate::metrics::METRIC_FLOW_TASK_COUNT; @@ -206,9 +207,17 @@ impl Flownode for FlowWorkerManager { }) .map(|r| (r, now, 1)) .collect_vec(); - self.handle_write_request(region_id.into(), rows) - .await + let batch_datatypes = insert_schema + .iter() + .map(from_proto_to_data_type) + .collect::, _>>() .map_err(to_meta_err)?; + self.handle_write_request(region_id.into(), rows, &batch_datatypes) + .await + .map_err(|err| { + common_telemetry::error!(err;"Failed to handle write request"); + to_meta_err(err) + })?; } Ok(Default::default()) } diff --git a/src/flow/src/adapter/node_context.rs b/src/flow/src/adapter/node_context.rs index 26e1a6483ab8..990fdd129797 100644 --- a/src/flow/src/adapter/node_context.rs +++ b/src/flow/src/adapter/node_context.rs @@ -19,6 +19,7 @@ use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; use common_telemetry::trace; +use datatypes::prelude::ConcreteDataType; use session::context::QueryContext; use snafu::{OptionExt, ResultExt}; use table::metadata::TableId; @@ -131,7 +132,11 @@ impl SourceSender { } /// return number of rows it actual send(including what's in the buffer) - pub async fn send_rows(&self, rows: Vec) -> Result { + pub async fn send_rows( + &self, + rows: Vec, + batch_datatypes: &[ConcreteDataType], + ) -> Result { METRIC_FLOW_INPUT_BUF_SIZE.add(rows.len() as _); while self.send_buf_row_cnt.load(Ordering::SeqCst) >= BATCH_SIZE * 4 { tokio::task::yield_now().await; @@ -139,8 +144,11 @@ impl SourceSender { // row count metrics is approx so relaxed order is ok self.send_buf_row_cnt .fetch_add(rows.len(), Ordering::SeqCst); - let batch = Batch::try_from_rows(rows.into_iter().map(|(row, _, _)| row).collect()) - .context(EvalSnafu)?; + let batch = Batch::try_from_rows_with_types( + rows.into_iter().map(|(row, _, _)| row).collect(), + batch_datatypes, + ) + .context(EvalSnafu)?; common_telemetry::trace!("Send one batch to worker with {} rows", batch.row_count()); self.send_buf_tx.send(batch).await.map_err(|e| { crate::error::InternalSnafu { @@ -157,14 +165,19 @@ impl FlownodeContext { /// return number of rows it actual send(including what's in the buffer) /// /// TODO(discord9): make this concurrent - pub async fn send(&self, table_id: TableId, rows: Vec) -> Result { + pub async fn send( + &self, + table_id: TableId, + rows: Vec, + batch_datatypes: &[ConcreteDataType], + ) -> Result { let sender = self .source_sender .get(&table_id) .with_context(|| TableNotFoundSnafu { name: table_id.to_string(), })?; - sender.send_rows(rows).await + sender.send_rows(rows, batch_datatypes).await } /// flush all sender's buf diff --git a/src/flow/src/adapter/util.rs b/src/flow/src/adapter/util.rs index a1d2895ba3be..f2a29bec8e9e 100644 --- a/src/flow/src/adapter/util.rs +++ b/src/flow/src/adapter/util.rs @@ -16,12 +16,27 @@ use api::helper::ColumnDataTypeWrapper; use api::v1::column_def::options_from_column_schema; use api::v1::{ColumnDataType, ColumnDataTypeExtension, SemanticType}; use common_error::ext::BoxedError; +use datatypes::prelude::ConcreteDataType; use datatypes::schema::ColumnSchema; use itertools::Itertools; use snafu::ResultExt; use crate::error::{Error, ExternalSnafu}; +pub fn from_proto_to_data_type( + column_schema: &api::v1::ColumnSchema, +) -> Result { + let wrapper = ColumnDataTypeWrapper::try_new( + column_schema.datatype, + column_schema.datatype_extension.clone(), + ) + .map_err(BoxedError::new) + .context(ExternalSnafu)?; + let cdt = ConcreteDataType::from(wrapper); + + Ok(cdt) +} + /// convert `ColumnSchema` lists to it's corresponding proto type pub fn column_schemas_to_proto( column_schemas: Vec, diff --git a/src/flow/src/compute/render.rs b/src/flow/src/compute/render.rs index 94f00a182921..46ac7e8a1d5e 100644 --- a/src/flow/src/compute/render.rs +++ b/src/flow/src/compute/render.rs @@ -30,7 +30,7 @@ use crate::compute::types::{Collection, CollectionBundle, ErrCollector, Toff}; use crate::error::{Error, InvalidQuerySnafu, NotImplementedSnafu}; use crate::expr::{self, Batch, GlobalId, LocalId}; use crate::plan::{Plan, TypedPlan}; -use crate::repr::{self, DiffRow}; +use crate::repr::{self, DiffRow, RelationType}; mod map; mod reduce; @@ -124,10 +124,10 @@ impl Context<'_, '_> { /// Like `render_plan` but in Batch Mode pub fn render_plan_batch(&mut self, plan: TypedPlan) -> Result, Error> { match plan.plan { - Plan::Constant { rows } => Ok(self.render_constant_batch(rows)), + Plan::Constant { rows } => Ok(self.render_constant_batch(rows, &plan.schema.typ)), Plan::Get { id } => self.get_batch_by_id(id), Plan::Let { id, value, body } => self.eval_batch_let(id, value, body), - Plan::Mfp { input, mfp } => self.render_mfp_batch(input, mfp), + Plan::Mfp { input, mfp } => self.render_mfp_batch(input, mfp, &plan.schema.typ), Plan::Reduce { input, key_val_plan, @@ -172,7 +172,11 @@ impl Context<'_, '_> { /// render Constant, take all rows that have a timestamp not greater than the current time /// This function is primarily used for testing /// Always assume input is sorted by timestamp - pub fn render_constant_batch(&mut self, rows: Vec) -> CollectionBundle { + pub fn render_constant_batch( + &mut self, + rows: Vec, + output_type: &RelationType, + ) -> CollectionBundle { let (send_port, recv_port) = self.df.make_edge::<_, Toff>("constant_batch"); let mut per_time: BTreeMap> = Default::default(); for (key, group) in &rows.into_iter().group_by(|(_row, ts, _diff)| *ts) { @@ -185,6 +189,8 @@ impl Context<'_, '_> { let scheduler_inner = scheduler.clone(); let err_collector = self.err_collector.clone(); + let output_type = output_type.clone(); + let subgraph_id = self.df .add_subgraph_source("ConstantBatch", send_port, move |_ctx, send_port| { @@ -199,7 +205,14 @@ impl Context<'_, '_> { not_great_than_now.into_iter().for_each(|(_ts, rows)| { err_collector.run(|| { let rows = rows.into_iter().map(|(row, _ts, _diff)| row).collect(); - let batch = Batch::try_from_rows(rows)?; + let batch = Batch::try_from_rows_with_types( + rows, + &output_type + .column_types + .iter() + .map(|ty| ty.scalar_type().clone()) + .collect_vec(), + )?; send_port.give(vec![batch]); Ok(()) }); diff --git a/src/flow/src/compute/render/map.rs b/src/flow/src/compute/render/map.rs index 416652328401..059e93fe3956 100644 --- a/src/flow/src/compute/render/map.rs +++ b/src/flow/src/compute/render/map.rs @@ -25,7 +25,7 @@ use crate::compute::types::{Arranged, Collection, CollectionBundle, ErrCollector use crate::error::{Error, PlanSnafu}; use crate::expr::{Batch, EvalError, MapFilterProject, MfpPlan, ScalarExpr}; use crate::plan::TypedPlan; -use crate::repr::{self, DiffRow, KeyValDiffRow, Row}; +use crate::repr::{self, DiffRow, KeyValDiffRow, RelationType, Row}; use crate::utils::ArrangeHandler; impl Context<'_, '_> { @@ -34,6 +34,7 @@ impl Context<'_, '_> { &mut self, input: Box, mfp: MapFilterProject, + _output_type: &RelationType, ) -> Result, Error> { let input = self.render_plan_batch(*input)?; diff --git a/src/flow/src/compute/render/reduce.rs b/src/flow/src/compute/render/reduce.rs index ee20f8ca783f..1d0689c4032f 100644 --- a/src/flow/src/compute/render/reduce.rs +++ b/src/flow/src/compute/render/reduce.rs @@ -87,6 +87,8 @@ impl Context<'_, '_> { })?; let key_val_plan = key_val_plan.clone(); + let output_type = output_type.clone(); + let now = self.compute_state.current_time_ref(); let err_collector = self.err_collector.clone(); @@ -118,6 +120,7 @@ impl Context<'_, '_> { src_data, &key_val_plan, &accum_plan, + &output_type, SubgraphArg { now, err_collector: &err_collector, @@ -354,6 +357,7 @@ fn reduce_batch_subgraph( src_data: impl IntoIterator, key_val_plan: &KeyValPlan, accum_plan: &AccumulablePlan, + output_type: &RelationType, SubgraphArg { now, err_collector, @@ -535,17 +539,13 @@ fn reduce_batch_subgraph( // this output part is not supposed to be resource intensive // (because for every batch there wouldn't usually be as many output row?), // so we can do some costly operation here - let output_types = all_output_dict.first_entry().map(|entry| { - entry - .key() - .iter() - .chain(entry.get().iter()) - .map(|v| v.data_type()) - .collect::>() - }); + let output_types = output_type + .column_types + .iter() + .map(|t| t.scalar_type.clone()) + .collect_vec(); - if let Some(output_types) = output_types { - err_collector.run(|| { + err_collector.run(|| { let column_cnt = output_types.len(); let row_cnt = all_output_dict.len(); @@ -585,7 +585,6 @@ fn reduce_batch_subgraph( Ok(()) }); - } } /// reduce subgraph, reduce the input data into a single row @@ -1516,7 +1515,9 @@ mod test { let mut ctx = harness_test_ctx(&mut df, &mut state); let rows = vec![ - (Row::new(vec![1i64.into()]), 1, 1), + (Row::new(vec![Value::Null]), -1, 1), + (Row::new(vec![1i64.into()]), 0, 1), + (Row::new(vec![Value::Null]), 1, 1), (Row::new(vec![2i64.into()]), 2, 1), (Row::new(vec![3i64.into()]), 3, 1), (Row::new(vec![1i64.into()]), 4, 1), @@ -1558,13 +1559,15 @@ mod test { Box::new(input_plan.with_types(typ.into_unnamed())), &key_val_plan, &reduce_plan, - &RelationType::empty(), + &RelationType::new(vec![ColumnType::new(CDT::int64_datatype(), true)]), ) .unwrap(); { let now_inner = now.clone(); let expected = BTreeMap::>::from([ + (-1, vec![]), + (0, vec![1i64]), (1, vec![1i64]), (2, vec![3i64]), (3, vec![6i64]), @@ -1581,7 +1584,11 @@ mod test { if let Some(expected) = expected.get(&now) { let batch = expected.iter().map(|v| Value::from(*v)).collect_vec(); - let batch = Batch::try_from_rows(vec![batch.into()]).unwrap(); + let batch = Batch::try_from_rows_with_types( + vec![batch.into()], + &[CDT::int64_datatype()], + ) + .unwrap(); assert_eq!(res.first(), Some(&batch)); } }); diff --git a/src/flow/src/expr.rs b/src/flow/src/expr.rs index 2e6019ba4ca7..5dde62b43a69 100644 --- a/src/flow/src/expr.rs +++ b/src/flow/src/expr.rs @@ -24,7 +24,7 @@ mod scalar; mod signature; use arrow::compute::FilterBuilder; -use datatypes::prelude::DataType; +use datatypes::prelude::{ConcreteDataType, DataType}; use datatypes::value::Value; use datatypes::vectors::{BooleanVector, Helper, VectorRef}; pub(crate) use df_func::{DfScalarFunction, RawDfScalarFn}; @@ -85,16 +85,18 @@ impl Default for Batch { } impl Batch { - pub fn try_from_rows(rows: Vec) -> Result { + /// Get batch from rows, will try best to determine data type + pub fn try_from_rows_with_types( + rows: Vec, + batch_datatypes: &[ConcreteDataType], + ) -> Result { if rows.is_empty() { return Ok(Self::empty()); } let len = rows.len(); - let mut builder = rows - .first() - .unwrap() + let mut builder = batch_datatypes .iter() - .map(|v| v.data_type().create_mutable_vector(len)) + .map(|ty| ty.create_mutable_vector(len)) .collect_vec(); for row in rows { ensure!( @@ -221,10 +223,25 @@ impl Batch { return Ok(()); } - let dts = if self.batch.is_empty() { - other.batch.iter().map(|v| v.data_type()).collect_vec() - } else { - self.batch.iter().map(|v| v.data_type()).collect_vec() + let dts = { + let max_len = self.batch.len().max(other.batch.len()); + let mut dts = Vec::with_capacity(max_len); + for i in 0..max_len { + if let Some(v) = self.batch().get(i) + && !v.data_type().is_null() + { + dts.push(v.data_type()) + } else if let Some(v) = other.batch().get(i) + && !v.data_type().is_null() + { + dts.push(v.data_type()) + } else { + // both are null, so we will push null type + dts.push(datatypes::prelude::ConcreteDataType::null_datatype()) + } + } + + dts }; let batch_builders = dts diff --git a/src/flow/src/expr/linear.rs b/src/flow/src/expr/linear.rs index 373e467aba1b..f96d7827b6bd 100644 --- a/src/flow/src/expr/linear.rs +++ b/src/flow/src/expr/linear.rs @@ -908,20 +908,33 @@ mod test { .unwrap() .unwrap(); assert_eq!(ret, Row::pack(vec![Value::from(false), Value::from(true)])); - + let ty = [ + ConcreteDataType::int32_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int32_datatype(), + ]; // batch mode - let mut batch = Batch::try_from_rows(vec![Row::from(vec![ - Value::from(4), - Value::from(2), - Value::from(3), - ])]) + let mut batch = Batch::try_from_rows_with_types( + vec![Row::from(vec![ + Value::from(4), + Value::from(2), + Value::from(3), + ])], + &ty, + ) .unwrap(); let ret = safe_mfp.eval_batch_into(&mut batch).unwrap(); assert_eq!( ret, - Batch::try_from_rows(vec![Row::from(vec![Value::from(false), Value::from(true)])]) - .unwrap() + Batch::try_from_rows_with_types( + vec![Row::from(vec![Value::from(false), Value::from(true)])], + &[ + ConcreteDataType::boolean_datatype(), + ConcreteDataType::boolean_datatype(), + ], + ) + .unwrap() ); } @@ -956,7 +969,15 @@ mod test { .unwrap(); assert_eq!(ret, None); - let mut input1_batch = Batch::try_from_rows(vec![Row::new(input1)]).unwrap(); + let input_type = [ + ConcreteDataType::int32_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::string_datatype(), + ]; + + let mut input1_batch = + Batch::try_from_rows_with_types(vec![Row::new(input1)], &input_type).unwrap(); let ret_batch = safe_mfp.eval_batch_into(&mut input1_batch).unwrap(); assert_eq!( ret_batch, @@ -974,7 +995,8 @@ mod test { .unwrap(); assert_eq!(ret, Some(Row::pack(vec![Value::from(11)]))); - let mut input2_batch = Batch::try_from_rows(vec![Row::new(input2)]).unwrap(); + let mut input2_batch = + Batch::try_from_rows_with_types(vec![Row::new(input2)], &input_type).unwrap(); let ret_batch = safe_mfp.eval_batch_into(&mut input2_batch).unwrap(); assert_eq!( ret_batch, @@ -1027,7 +1049,14 @@ mod test { let ret = safe_mfp.evaluate_into(&mut input1.clone(), &mut Row::empty()); assert!(matches!(ret, Err(EvalError::InvalidArgument { .. }))); - let mut input1_batch = Batch::try_from_rows(vec![Row::new(input1)]).unwrap(); + let input_type = [ + ConcreteDataType::int64_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int32_datatype(), + ]; + let mut input1_batch = + Batch::try_from_rows_with_types(vec![Row::new(input1)], &input_type).unwrap(); let ret_batch = safe_mfp.eval_batch_into(&mut input1_batch); assert!(matches!(ret_batch, Err(EvalError::InvalidArgument { .. }))); @@ -1037,7 +1066,13 @@ mod test { .unwrap(); assert_eq!(ret, Some(Row::new(input2.clone()))); - let input2_batch = Batch::try_from_rows(vec![Row::new(input2)]).unwrap(); + let input_type = [ + ConcreteDataType::int64_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int32_datatype(), + ]; + let input2_batch = + Batch::try_from_rows_with_types(vec![Row::new(input2)], &input_type).unwrap(); let ret_batch = safe_mfp.eval_batch_into(&mut input2_batch.clone()).unwrap(); assert_eq!(ret_batch, input2_batch); @@ -1047,7 +1082,8 @@ mod test { .unwrap(); assert_eq!(ret, None); - let input3_batch = Batch::try_from_rows(vec![Row::new(input3)]).unwrap(); + let input3_batch = + Batch::try_from_rows_with_types(vec![Row::new(input3)], &input_type).unwrap(); let ret_batch = safe_mfp.eval_batch_into(&mut input3_batch.clone()).unwrap(); assert_eq!( ret_batch, @@ -1083,7 +1119,13 @@ mod test { let ret = safe_mfp.evaluate_into(&mut input1.clone(), &mut Row::empty()); assert_eq!(ret.unwrap(), Some(Row::new(vec![Value::from(false)]))); - let mut input1_batch = Batch::try_from_rows(vec![Row::new(input1)]).unwrap(); + let input_type = [ + ConcreteDataType::int32_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int32_datatype(), + ]; + let mut input1_batch = + Batch::try_from_rows_with_types(vec![Row::new(input1)], &input_type).unwrap(); let ret_batch = safe_mfp.eval_batch_into(&mut input1_batch).unwrap(); assert_eq!( diff --git a/tests/cases/standalone/common/flow/flow_basic.result b/tests/cases/standalone/common/flow/flow_basic.result index fa360a6de684..c70fe54fec19 100644 --- a/tests/cases/standalone/common/flow/flow_basic.result +++ b/tests/cases/standalone/common/flow/flow_basic.result @@ -390,6 +390,65 @@ GROUP BY Affected Rows: 0 +INSERT INTO + bytes_log +VALUES + (NULL, '2023-01-01 00:00:01'), + (300, '2023-01-01 00:00:29'); + +Affected Rows: 2 + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('find_approx_rate'); + ++--------------------------------------+ +| ADMIN FLUSH_FLOW('find_approx_rate') | ++--------------------------------------+ +| FLOW_FLUSHED | ++--------------------------------------+ + +SELECT + rate, + time_window +FROM + approx_rate; + ++------+---------------------+ +| rate | time_window | ++------+---------------------+ +| 0.0 | 2023-01-01T00:00:00 | ++------+---------------------+ + +INSERT INTO + bytes_log +VALUES + (NULL, '2022-01-01 00:00:01'), + (NULL, '2022-01-01 00:00:29'); + +Affected Rows: 2 + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('find_approx_rate'); + ++--------------------------------------+ +| ADMIN FLUSH_FLOW('find_approx_rate') | ++--------------------------------------+ +| FLOW_FLUSHED | ++--------------------------------------+ + +SELECT + rate, + time_window +FROM + approx_rate; + ++------+---------------------+ +| rate | time_window | ++------+---------------------+ +| | 2022-01-01T00:00:00 | +| 0.0 | 2023-01-01T00:00:00 | ++------+---------------------+ + INSERT INTO bytes_log VALUES @@ -416,6 +475,8 @@ FROM +-------------------+---------------------+ | rate | time_window | +-------------------+---------------------+ +| | 2022-01-01T00:00:00 | +| 0.0 | 2023-01-01T00:00:00 | | 6.633333333333334 | 2025-01-01T00:00:00 | +-------------------+---------------------+ @@ -445,6 +506,8 @@ FROM +--------------------+---------------------+ | rate | time_window | +--------------------+---------------------+ +| | 2022-01-01T00:00:00 | +| 0.0 | 2023-01-01T00:00:00 | | 6.633333333333334 | 2025-01-01T00:00:00 | | 1.6666666666666667 | 2025-01-01T00:00:30 | +--------------------+---------------------+ @@ -992,6 +1055,7 @@ CREATE TABLE requests_without_ip ( service_name STRING, val INT, ts TIMESTAMP TIME INDEX, + PRIMARY KEY(service_name) ); Affected Rows: 0 @@ -1009,12 +1073,12 @@ Affected Rows: 0 INSERT INTO requests VALUES - ("svc1", "10.0.0.1", 100, "2024-10-18 19:00:00"), + (NULL, "10.0.0.1", 100, "2024-10-18 19:00:00"), ("svc1", "10.0.0.2", 100, "2024-10-18 19:00:00"), - ("svc1", "10.0.0.1", 200, "2024-10-18 19:00:30"), + (NULL, "10.0.0.1", 200, "2024-10-18 19:00:30"), ("svc1", "10.0.0.2", 200, "2024-10-18 19:00:30"), - ("svc1", "10.0.0.1", 300, "2024-10-18 19:01:00"), - ("svc1", "10.0.0.2", 100, "2024-10-18 19:01:01"), + (NULL, "10.0.0.1", 300, "2024-10-18 19:01:00"), + (NULL, "10.0.0.2", 100, "2024-10-18 19:01:01"), ("svc1", "10.0.0.1", 400, "2024-10-18 19:01:30"), ("svc1", "10.0.0.2", 200, "2024-10-18 19:01:31"); @@ -1037,10 +1101,12 @@ FROM +--------------+-----+---------------------+ | service_name | val | ts | +--------------+-----+---------------------+ +| | 100 | 2024-10-18T19:00:00 | +| | 200 | 2024-10-18T19:00:30 | +| | 300 | 2024-10-18T19:01:00 | +| | 100 | 2024-10-18T19:01:01 | | svc1 | 100 | 2024-10-18T19:00:00 | | svc1 | 200 | 2024-10-18T19:00:30 | -| svc1 | 300 | 2024-10-18T19:01:00 | -| svc1 | 100 | 2024-10-18T19:01:01 | | svc1 | 400 | 2024-10-18T19:01:30 | | svc1 | 200 | 2024-10-18T19:01:31 | +--------------+-----+---------------------+ @@ -1057,6 +1123,106 @@ FROM | 1 | +--------------+ +INSERT INTO + requests +VALUES + (null, "10.0.0.1", 100, "2024-10-19 19:00:00"), + (null, "10.0.0.2", 100, "2024-10-19 19:00:00"), + (null, "10.0.0.1", 200, "2024-10-19 19:00:30"), + (null, "10.0.0.2", 200, "2024-10-19 19:00:30"), + (null, "10.0.0.1", 300, "2024-10-19 19:01:00"), + (null, "10.0.0.2", 100, "2024-10-19 19:01:01"), + (null, "10.0.0.1", 400, "2024-10-19 19:01:30"), + (null, "10.0.0.2", 200, "2024-10-19 19:01:31"); + +Affected Rows: 8 + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('requests_long_term'); + ++----------------------------------------+ +| ADMIN FLUSH_FLOW('requests_long_term') | ++----------------------------------------+ +| FLOW_FLUSHED | ++----------------------------------------+ + +SELECT + * +FROM + requests_without_ip; + ++--------------+-----+---------------------+ +| service_name | val | ts | ++--------------+-----+---------------------+ +| | 100 | 2024-10-18T19:00:00 | +| | 200 | 2024-10-18T19:00:30 | +| | 300 | 2024-10-18T19:01:00 | +| | 100 | 2024-10-18T19:01:01 | +| | 100 | 2024-10-19T19:00:00 | +| | 200 | 2024-10-19T19:00:30 | +| | 300 | 2024-10-19T19:01:00 | +| | 100 | 2024-10-19T19:01:01 | +| | 400 | 2024-10-19T19:01:30 | +| | 200 | 2024-10-19T19:01:31 | +| svc1 | 100 | 2024-10-18T19:00:00 | +| svc1 | 200 | 2024-10-18T19:00:30 | +| svc1 | 400 | 2024-10-18T19:01:30 | +| svc1 | 200 | 2024-10-18T19:01:31 | ++--------------+-----+---------------------+ + +INSERT INTO + requests +VALUES + ("svc2", "10.0.0.1", 100, "2024-10-18 19:00:00"), + ("svc2", "10.0.0.2", 100, "2024-10-18 19:00:00"), + ("svc2", "10.0.0.1", 200, "2024-10-18 19:00:30"), + ("svc2", "10.0.0.2", 200, "2024-10-18 19:00:30"), + ("svc2", "10.0.0.1", 300, "2024-10-18 19:01:00"), + ("svc2", "10.0.0.2", 100, "2024-10-18 19:01:01"), + ("svc2", "10.0.0.1", 400, "2024-10-18 19:01:30"), + ("svc2", "10.0.0.2", 200, "2024-10-18 19:01:31"); + +Affected Rows: 8 + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('requests_long_term'); + ++----------------------------------------+ +| ADMIN FLUSH_FLOW('requests_long_term') | ++----------------------------------------+ +| FLOW_FLUSHED | ++----------------------------------------+ + +SELECT + * +FROM + requests_without_ip; + ++--------------+-----+---------------------+ +| service_name | val | ts | ++--------------+-----+---------------------+ +| | 100 | 2024-10-18T19:00:00 | +| | 200 | 2024-10-18T19:00:30 | +| | 300 | 2024-10-18T19:01:00 | +| | 100 | 2024-10-18T19:01:01 | +| | 100 | 2024-10-19T19:00:00 | +| | 200 | 2024-10-19T19:00:30 | +| | 300 | 2024-10-19T19:01:00 | +| | 100 | 2024-10-19T19:01:01 | +| | 400 | 2024-10-19T19:01:30 | +| | 200 | 2024-10-19T19:01:31 | +| svc1 | 100 | 2024-10-18T19:00:00 | +| svc1 | 200 | 2024-10-18T19:00:30 | +| svc1 | 400 | 2024-10-18T19:01:30 | +| svc1 | 200 | 2024-10-18T19:01:31 | +| svc2 | 100 | 2024-10-18T19:00:00 | +| svc2 | 200 | 2024-10-18T19:00:30 | +| svc2 | 300 | 2024-10-18T19:01:00 | +| svc2 | 100 | 2024-10-18T19:01:01 | +| svc2 | 400 | 2024-10-18T19:01:30 | +| svc2 | 200 | 2024-10-18T19:01:31 | ++--------------+-----+---------------------+ + DROP FLOW requests_long_term; Affected Rows: 0 diff --git a/tests/cases/standalone/common/flow/flow_basic.sql b/tests/cases/standalone/common/flow/flow_basic.sql index 8946c014be36..74abbc85df22 100644 --- a/tests/cases/standalone/common/flow/flow_basic.sql +++ b/tests/cases/standalone/common/flow/flow_basic.sql @@ -214,6 +214,36 @@ from GROUP BY time_window; +INSERT INTO + bytes_log +VALUES + (NULL, '2023-01-01 00:00:01'), + (300, '2023-01-01 00:00:29'); + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('find_approx_rate'); + +SELECT + rate, + time_window +FROM + approx_rate; + +INSERT INTO + bytes_log +VALUES + (NULL, '2022-01-01 00:00:01'), + (NULL, '2022-01-01 00:00:29'); + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('find_approx_rate'); + +SELECT + rate, + time_window +FROM + approx_rate; + INSERT INTO bytes_log VALUES @@ -539,6 +569,7 @@ CREATE TABLE requests_without_ip ( service_name STRING, val INT, ts TIMESTAMP TIME INDEX, + PRIMARY KEY(service_name) ); CREATE FLOW requests_long_term SINK TO requests_without_ip AS @@ -552,12 +583,12 @@ FROM INSERT INTO requests VALUES - ("svc1", "10.0.0.1", 100, "2024-10-18 19:00:00"), + (NULL, "10.0.0.1", 100, "2024-10-18 19:00:00"), ("svc1", "10.0.0.2", 100, "2024-10-18 19:00:00"), - ("svc1", "10.0.0.1", 200, "2024-10-18 19:00:30"), + (NULL, "10.0.0.1", 200, "2024-10-18 19:00:30"), ("svc1", "10.0.0.2", 200, "2024-10-18 19:00:30"), - ("svc1", "10.0.0.1", 300, "2024-10-18 19:01:00"), - ("svc1", "10.0.0.2", 100, "2024-10-18 19:01:01"), + (NULL, "10.0.0.1", 300, "2024-10-18 19:01:00"), + (NULL, "10.0.0.2", 100, "2024-10-18 19:01:01"), ("svc1", "10.0.0.1", 400, "2024-10-18 19:01:30"), ("svc1", "10.0.0.2", 200, "2024-10-18 19:01:31"); @@ -575,6 +606,46 @@ SELECT FROM INFORMATION_SCHEMA.FLOWS; +INSERT INTO + requests +VALUES + (null, "10.0.0.1", 100, "2024-10-19 19:00:00"), + (null, "10.0.0.2", 100, "2024-10-19 19:00:00"), + (null, "10.0.0.1", 200, "2024-10-19 19:00:30"), + (null, "10.0.0.2", 200, "2024-10-19 19:00:30"), + (null, "10.0.0.1", 300, "2024-10-19 19:01:00"), + (null, "10.0.0.2", 100, "2024-10-19 19:01:01"), + (null, "10.0.0.1", 400, "2024-10-19 19:01:30"), + (null, "10.0.0.2", 200, "2024-10-19 19:01:31"); + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('requests_long_term'); + +SELECT + * +FROM + requests_without_ip; + +INSERT INTO + requests +VALUES + ("svc2", "10.0.0.1", 100, "2024-10-18 19:00:00"), + ("svc2", "10.0.0.2", 100, "2024-10-18 19:00:00"), + ("svc2", "10.0.0.1", 200, "2024-10-18 19:00:30"), + ("svc2", "10.0.0.2", 200, "2024-10-18 19:00:30"), + ("svc2", "10.0.0.1", 300, "2024-10-18 19:01:00"), + ("svc2", "10.0.0.2", 100, "2024-10-18 19:01:01"), + ("svc2", "10.0.0.1", 400, "2024-10-18 19:01:30"), + ("svc2", "10.0.0.2", 200, "2024-10-18 19:01:31"); + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('requests_long_term'); + +SELECT + * +FROM + requests_without_ip; + DROP FLOW requests_long_term; DROP TABLE requests_without_ip; From 6bf574f098cac4950ab52eaf6b11a50caf14824b Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Thu, 19 Dec 2024 19:23:01 +0800 Subject: [PATCH 04/12] fix: auto created table ttl check (#5203) * fix: auto created table ttl check * tests: with hint --- src/operator/src/insert.rs | 6 ++++++ tests-integration/tests/grpc.rs | 34 +++++++++++++++++++++++++++++++++ 2 files changed, 40 insertions(+) diff --git a/src/operator/src/insert.rs b/src/operator/src/insert.rs index ec01b329457f..466dde5425c1 100644 --- a/src/operator/src/insert.rs +++ b/src/operator/src/insert.rs @@ -576,6 +576,9 @@ impl Inserter { for table in tables { let table_info = table.table_info(); + if table_info.is_ttl_instant_table() { + instant_table_ids.insert(table_info.table_id()); + } table_name_to_ids.insert(table_info.name.clone(), table_info.table_id()); } } @@ -596,6 +599,9 @@ impl Inserter { .create_physical_table(create_table, ctx, statement_executor) .await?; let table_info = table.table_info(); + if table_info.is_ttl_instant_table() { + instant_table_ids.insert(table_info.table_id()); + } table_name_to_ids.insert(table_info.name.clone(), table_info.table_id()); } for alter_expr in alter_tables.into_iter() { diff --git a/tests-integration/tests/grpc.rs b/tests-integration/tests/grpc.rs index 8b91ed55d520..74c8a6c0f73d 100644 --- a/tests-integration/tests/grpc.rs +++ b/tests-integration/tests/grpc.rs @@ -444,6 +444,40 @@ async fn insert_with_hints_and_assert(db: &Database) { +-------+-------------------------------------+\ "; assert_eq!(pretty, expected); + + // testing data with ttl=instant and auto_create_table = true can be handled correctly + let (expected_host_col, expected_cpu_col, expected_mem_col, expected_ts_col) = expect_data(); + + let request = InsertRequest { + table_name: "demo1".to_string(), + columns: vec![ + expected_host_col.clone(), + expected_cpu_col.clone(), + expected_mem_col.clone(), + expected_ts_col.clone(), + ], + row_count: 4, + }; + let result = db + .insert_with_hints( + InsertRequests { + inserts: vec![request], + }, + &[("auto_create_table", "true"), ("ttl", "instant")], + ) + .await; + assert_eq!(result.unwrap(), 0); + + // check table is empty + let output = db.sql("SELECT * FROM demo1").await.unwrap(); + + let record_batches = match output.data { + OutputData::RecordBatches(record_batches) => record_batches, + OutputData::Stream(stream) => RecordBatches::try_collect(stream).await.unwrap(), + OutputData::AffectedRows(_) => unreachable!(), + }; + + assert!(record_batches.iter().all(|r| r.num_rows() == 0)); } async fn insert_and_assert(db: &Database) { From a578eea801459100596fa9d5be7752cccddaddec Mon Sep 17 00:00:00 2001 From: LFC <990479+MichaelScofield@users.noreply.github.com> Date: Fri, 20 Dec 2024 10:45:53 +0800 Subject: [PATCH 05/12] ci: install latest protobuf in dev-builder image (#5196) --- docker/dev-builder/ubuntu/Dockerfile | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/docker/dev-builder/ubuntu/Dockerfile b/docker/dev-builder/ubuntu/Dockerfile index 733bfdab62f5..9f16161ac0c2 100644 --- a/docker/dev-builder/ubuntu/Dockerfile +++ b/docker/dev-builder/ubuntu/Dockerfile @@ -15,8 +15,8 @@ RUN apt-get update && \ RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y \ libssl-dev \ tzdata \ - protobuf-compiler \ curl \ + unzip \ ca-certificates \ git \ build-essential \ @@ -24,6 +24,20 @@ RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y \ python3.10 \ python3.10-dev +ARG TARGETPLATFORM +RUN echo "target platform: $TARGETPLATFORM" + +# Install protobuf, because the one in the apt is too old (v3.12). +RUN if [ "$TARGETPLATFORM" = "linux/arm64" ]; then \ + curl -OL https://github.com/protocolbuffers/protobuf/releases/download/v29.1/protoc-29.1-linux-aarch_64.zip && \ + unzip protoc-29.1-linux-aarch_64.zip -d protoc3; \ +elif [ "$TARGETPLATFORM" = "linux/amd64" ]; then \ + curl -OL https://github.com/protocolbuffers/protobuf/releases/download/v29.1/protoc-29.1-linux-x86_64.zip && \ + unzip protoc-29.1-linux-x86_64.zip -d protoc3; \ +fi +RUN mv protoc3/bin/* /usr/local/bin/ +RUN mv protoc3/include/* /usr/local/include/ + # https://github.com/GreptimeTeam/greptimedb/actions/runs/10935485852/job/30357457188#step:3:7106 # `aws-lc-sys` require gcc >= 10.3.0 to work, hence alias to use gcc-10 RUN apt-get remove -y gcc-9 g++-9 cpp-9 && \ @@ -49,7 +63,7 @@ RUN apt-get -y purge python3.8 && \ # wildcard here. However, that requires the git's config files and the submodules all owned by the very same user. # It's troublesome to do this since the dev build runs in Docker, which is under user "root"; while outside the Docker, # it can be a different user that have prepared the submodules. -RUN git config --global --add safe.directory * +RUN git config --global --add safe.directory '*' # Install Python dependencies. COPY $DOCKER_BUILD_ROOT/docker/python/requirements.txt /etc/greptime/requirements.txt From 1910d71cb35749f8beae5d56d0543b4b9457054f Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Fri, 20 Dec 2024 14:58:51 +0800 Subject: [PATCH 06/12] chore: adjust fuzz tests cfg (#5207) --- .../setup-greptimedb-cluster/with-minio-and-cache.yaml | 4 ++-- .github/actions/setup-greptimedb-cluster/with-minio.yaml | 4 ++-- .../actions/setup-greptimedb-cluster/with-remote-wal.yaml | 4 ++-- tests-fuzz/src/utils.rs | 6 +++--- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/.github/actions/setup-greptimedb-cluster/with-minio-and-cache.yaml b/.github/actions/setup-greptimedb-cluster/with-minio-and-cache.yaml index 8f99c242f4bf..864e59c4bc7b 100644 --- a/.github/actions/setup-greptimedb-cluster/with-minio-and-cache.yaml +++ b/.github/actions/setup-greptimedb-cluster/with-minio-and-cache.yaml @@ -5,7 +5,7 @@ meta: [datanode] [datanode.client] - timeout = "60s" + timeout = "120s" datanode: configData: |- [runtime] @@ -21,7 +21,7 @@ frontend: global_rt_size = 4 [meta_client] - ddl_timeout = "60s" + ddl_timeout = "120s" objectStorage: s3: bucket: default diff --git a/.github/actions/setup-greptimedb-cluster/with-minio.yaml b/.github/actions/setup-greptimedb-cluster/with-minio.yaml index b8121374ce03..2e6a7b930eb1 100644 --- a/.github/actions/setup-greptimedb-cluster/with-minio.yaml +++ b/.github/actions/setup-greptimedb-cluster/with-minio.yaml @@ -5,7 +5,7 @@ meta: [datanode] [datanode.client] - timeout = "60s" + timeout = "120s" datanode: configData: |- [runtime] @@ -17,7 +17,7 @@ frontend: global_rt_size = 4 [meta_client] - ddl_timeout = "60s" + ddl_timeout = "120s" objectStorage: s3: bucket: default diff --git a/.github/actions/setup-greptimedb-cluster/with-remote-wal.yaml b/.github/actions/setup-greptimedb-cluster/with-remote-wal.yaml index 043b9fe43f85..a97f921f8c26 100644 --- a/.github/actions/setup-greptimedb-cluster/with-remote-wal.yaml +++ b/.github/actions/setup-greptimedb-cluster/with-remote-wal.yaml @@ -11,7 +11,7 @@ meta: [datanode] [datanode.client] - timeout = "60s" + timeout = "120s" datanode: configData: |- [runtime] @@ -28,7 +28,7 @@ frontend: global_rt_size = 4 [meta_client] - ddl_timeout = "60s" + ddl_timeout = "120s" objectStorage: s3: bucket: default diff --git a/tests-fuzz/src/utils.rs b/tests-fuzz/src/utils.rs index 84222f6d5a58..8e5daef4eb1d 100644 --- a/tests-fuzz/src/utils.rs +++ b/tests-fuzz/src/utils.rs @@ -141,10 +141,10 @@ macro_rules! make_get_from_env_helper { } make_get_from_env_helper!(GT_FUZZ_INPUT_MAX_ALTER_ACTIONS, 256); -make_get_from_env_helper!(GT_FUZZ_INPUT_MAX_INSERT_ACTIONS, 8); +make_get_from_env_helper!(GT_FUZZ_INPUT_MAX_INSERT_ACTIONS, 4); make_get_from_env_helper!(GT_FUZZ_INPUT_MAX_ROWS, 512); -make_get_from_env_helper!(GT_FUZZ_INPUT_MAX_TABLES, 64); -make_get_from_env_helper!(GT_FUZZ_INPUT_MAX_COLUMNS, 32); +make_get_from_env_helper!(GT_FUZZ_INPUT_MAX_TABLES, 32); +make_get_from_env_helper!(GT_FUZZ_INPUT_MAX_COLUMNS, 16); /// Retrieves a value from the environment variables /// or returns a default value if the environment variable is not set. From 3d4121aefbbb42de27b7039959b2219a4222bfc6 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 20 Dec 2024 14:59:44 +0800 Subject: [PATCH 07/12] feat(bloom-filter): add memory control for creator (#5185) * feat(bloom-filter): add memory control for creator Signed-off-by: Zhenchi * refactor: remove meaningless buf Signed-off-by: Zhenchi * feat: add codec for intermediate Signed-off-by: Zhenchi --------- Signed-off-by: Zhenchi --- Cargo.lock | 1 + src/index/Cargo.toml | 1 + src/index/src/bloom_filter/creator.rs | 171 +++++----- .../bloom_filter/creator/finalize_segment.rs | 293 ++++++++++++++++++ .../creator/intermediate_codec.rs | 248 +++++++++++++++ src/index/src/bloom_filter/error.rs | 19 +- src/index/src/error.rs | 48 +++ .../create/sort => }/external_provider.rs | 23 +- src/index/src/inverted_index/create/sort.rs | 1 - .../create/sort/external_sort.rs | 23 +- src/index/src/inverted_index/error.rs | 8 + src/index/src/lib.rs | 2 + src/mito2/src/sst/index/intermediate.rs | 41 ++- .../inverted_index/creator/temp_provider.rs | 36 ++- 14 files changed, 768 insertions(+), 147 deletions(-) create mode 100644 src/index/src/bloom_filter/creator/finalize_segment.rs create mode 100644 src/index/src/bloom_filter/creator/intermediate_codec.rs create mode 100644 src/index/src/error.rs rename src/index/src/{inverted_index/create/sort => }/external_provider.rs (58%) diff --git a/Cargo.lock b/Cargo.lock index c23acf60636d..8f5b1037193b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5270,6 +5270,7 @@ dependencies = [ name = "index" version = "0.12.0" dependencies = [ + "async-stream", "async-trait", "asynchronous-codec", "bytemuck", diff --git a/src/index/Cargo.toml b/src/index/Cargo.toml index f46c64a17606..f91a8eac9708 100644 --- a/src/index/Cargo.toml +++ b/src/index/Cargo.toml @@ -8,6 +8,7 @@ license.workspace = true workspace = true [dependencies] +async-stream.workspace = true async-trait.workspace = true asynchronous-codec = "0.7.0" bytemuck.workspace = true diff --git a/src/index/src/bloom_filter/creator.rs b/src/index/src/bloom_filter/creator.rs index b3c95d3a7626..04d2edc3d1b3 100644 --- a/src/index/src/bloom_filter/creator.rs +++ b/src/index/src/bloom_filter/creator.rs @@ -12,21 +12,27 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod finalize_segment; +mod intermediate_codec; + use std::collections::HashSet; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::Arc; -use fastbloom::BloomFilter; -use futures::{AsyncWrite, AsyncWriteExt}; +use finalize_segment::FinalizedBloomFilterStorage; +use futures::{AsyncWrite, AsyncWriteExt, StreamExt}; use snafu::ResultExt; use super::error::{IoSnafu, SerdeJsonSnafu}; use crate::bloom_filter::error::Result; use crate::bloom_filter::{BloomFilterMeta, BloomFilterSegmentLocation, Bytes}; +use crate::external_provider::ExternalTempFileProvider; /// The seed used for the Bloom filter. -const SEED: u128 = 42; +pub const SEED: u128 = 42; /// The false positive rate of the Bloom filter. -const FALSE_POSITIVE_RATE: f64 = 0.01; +pub const FALSE_POSITIVE_RATE: f64 = 0.01; /// `BloomFilterCreator` is responsible for creating and managing bloom filters /// for a set of elements. It divides the rows into segments and creates @@ -58,6 +64,9 @@ pub struct BloomFilterCreator { /// Storage for finalized Bloom filters. finalized_bloom_filters: FinalizedBloomFilterStorage, + + /// Global memory usage of the bloom filter creator. + global_memory_usage: Arc, } impl BloomFilterCreator { @@ -66,7 +75,12 @@ impl BloomFilterCreator { /// # PANICS /// /// `rows_per_segment` <= 0 - pub fn new(rows_per_segment: usize) -> Self { + pub fn new( + rows_per_segment: usize, + intermediate_provider: Box, + global_memory_usage: Arc, + global_memory_usage_threshold: Option, + ) -> Self { assert!( rows_per_segment > 0, "rows_per_segment must be greater than 0" @@ -77,54 +91,67 @@ impl BloomFilterCreator { accumulated_row_count: 0, cur_seg_distinct_elems: HashSet::default(), cur_seg_distinct_elems_mem_usage: 0, - finalized_bloom_filters: FinalizedBloomFilterStorage::default(), + global_memory_usage: global_memory_usage.clone(), + finalized_bloom_filters: FinalizedBloomFilterStorage::new( + intermediate_provider, + global_memory_usage, + global_memory_usage_threshold, + ), } } /// Adds a row of elements to the bloom filter. If the number of accumulated rows /// reaches `rows_per_segment`, it finalizes the current segment. - pub fn push_row_elems(&mut self, elems: impl IntoIterator) { + pub async fn push_row_elems(&mut self, elems: impl IntoIterator) -> Result<()> { self.accumulated_row_count += 1; + + let mut mem_diff = 0; for elem in elems.into_iter() { let len = elem.len(); let is_new = self.cur_seg_distinct_elems.insert(elem); if is_new { - self.cur_seg_distinct_elems_mem_usage += len; + mem_diff += len; } } + self.cur_seg_distinct_elems_mem_usage += mem_diff; + self.global_memory_usage + .fetch_add(mem_diff, Ordering::Relaxed); if self.accumulated_row_count % self.rows_per_segment == 0 { - self.finalize_segment(); + self.finalize_segment().await?; } + + Ok(()) } /// Finalizes any remaining segments and writes the bloom filters and metadata to the provided writer. pub async fn finish(&mut self, mut writer: impl AsyncWrite + Unpin) -> Result<()> { if !self.cur_seg_distinct_elems.is_empty() { - self.finalize_segment(); + self.finalize_segment().await?; } let mut meta = BloomFilterMeta { rows_per_segment: self.rows_per_segment, - seg_count: self.finalized_bloom_filters.len(), row_count: self.accumulated_row_count, ..Default::default() }; - let mut buf = Vec::new(); - for segment in self.finalized_bloom_filters.drain() { - let slice = segment.bloom_filter.as_slice(); - buf.clear(); - write_u64_slice(&mut buf, slice); - writer.write_all(&buf).await.context(IoSnafu)?; + let mut segs = self.finalized_bloom_filters.drain().await?; + while let Some(segment) = segs.next().await { + let segment = segment?; + writer + .write_all(&segment.bloom_filter_bytes) + .await + .context(IoSnafu)?; - let size = buf.len(); + let size = segment.bloom_filter_bytes.len(); meta.bloom_filter_segments.push(BloomFilterSegmentLocation { offset: meta.bloom_filter_segments_size as _, size: size as _, elem_count: segment.element_count, }); meta.bloom_filter_segments_size += size; + meta.seg_count += 1; } let meta_bytes = serde_json::to_vec(&meta).context(SerdeJsonSnafu)?; @@ -145,91 +172,29 @@ impl BloomFilterCreator { self.cur_seg_distinct_elems_mem_usage + self.finalized_bloom_filters.memory_usage() } - fn finalize_segment(&mut self) { + async fn finalize_segment(&mut self) -> Result<()> { let elem_count = self.cur_seg_distinct_elems.len(); self.finalized_bloom_filters - .add(self.cur_seg_distinct_elems.drain(), elem_count); - self.cur_seg_distinct_elems_mem_usage = 0; - } -} - -/// Storage for finalized Bloom filters. -/// -/// TODO(zhongzc): Add support for storing intermediate bloom filters on disk to control memory usage. -#[derive(Debug, Default)] -struct FinalizedBloomFilterStorage { - /// Bloom filters that are stored in memory. - in_memory: Vec, -} - -impl FinalizedBloomFilterStorage { - fn memory_usage(&self) -> usize { - self.in_memory.iter().map(|s| s.size).sum() - } - - /// Adds a new finalized Bloom filter to the storage. - /// - /// TODO(zhongzc): Add support for flushing to disk. - fn add(&mut self, elems: impl IntoIterator, elem_count: usize) { - let mut bf = BloomFilter::with_false_pos(FALSE_POSITIVE_RATE) - .seed(&SEED) - .expected_items(elem_count); - for elem in elems.into_iter() { - bf.insert(&elem); - } + .add(self.cur_seg_distinct_elems.drain(), elem_count) + .await?; - let cbf = FinalizedBloomFilterSegment::new(bf, elem_count); - self.in_memory.push(cbf); - } - - fn len(&self) -> usize { - self.in_memory.len() - } - - fn drain(&mut self) -> impl Iterator + '_ { - self.in_memory.drain(..) - } -} - -/// A finalized Bloom filter segment. -#[derive(Debug)] -struct FinalizedBloomFilterSegment { - /// The underlying Bloom filter. - bloom_filter: BloomFilter, - - /// The number of elements in the Bloom filter. - element_count: usize, - - /// The occupied memory size of the Bloom filter. - size: usize, -} - -impl FinalizedBloomFilterSegment { - fn new(bloom_filter: BloomFilter, elem_count: usize) -> Self { - let memory_usage = std::mem::size_of_val(bloom_filter.as_slice()); - Self { - bloom_filter, - element_count: elem_count, - size: memory_usage, - } - } -} - -/// Writes a slice of `u64` to the buffer in little-endian order. -fn write_u64_slice(buf: &mut Vec, slice: &[u64]) { - buf.reserve(std::mem::size_of_val(slice)); - for &x in slice { - buf.extend_from_slice(&x.to_le_bytes()); + self.global_memory_usage + .fetch_sub(self.cur_seg_distinct_elems_mem_usage, Ordering::Relaxed); + self.cur_seg_distinct_elems_mem_usage = 0; + Ok(()) } } #[cfg(test)] mod tests { + use fastbloom::BloomFilter; use futures::io::Cursor; use super::*; + use crate::external_provider::MockExternalTempFileProvider; - fn u64_vec_from_bytes(bytes: &[u8]) -> Vec { + /// Converts a slice of bytes to a vector of `u64`. + pub fn u64_vec_from_bytes(bytes: &[u8]) -> Vec { bytes .chunks_exact(std::mem::size_of::()) .map(|chunk| u64::from_le_bytes(chunk.try_into().unwrap())) @@ -239,18 +204,32 @@ mod tests { #[tokio::test] async fn test_bloom_filter_creator() { let mut writer = Cursor::new(Vec::new()); - let mut creator = BloomFilterCreator::new(2); + let mut creator = BloomFilterCreator::new( + 2, + Box::new(MockExternalTempFileProvider::new()), + Arc::new(AtomicUsize::new(0)), + None, + ); - creator.push_row_elems(vec![b"a".to_vec(), b"b".to_vec()]); + creator + .push_row_elems(vec![b"a".to_vec(), b"b".to_vec()]) + .await + .unwrap(); assert!(creator.cur_seg_distinct_elems_mem_usage > 0); assert!(creator.memory_usage() > 0); - creator.push_row_elems(vec![b"c".to_vec(), b"d".to_vec()]); + creator + .push_row_elems(vec![b"c".to_vec(), b"d".to_vec()]) + .await + .unwrap(); // Finalize the first segment - assert!(creator.cur_seg_distinct_elems_mem_usage == 0); + assert_eq!(creator.cur_seg_distinct_elems_mem_usage, 0); assert!(creator.memory_usage() > 0); - creator.push_row_elems(vec![b"e".to_vec(), b"f".to_vec()]); + creator + .push_row_elems(vec![b"e".to_vec(), b"f".to_vec()]) + .await + .unwrap(); assert!(creator.cur_seg_distinct_elems_mem_usage > 0); assert!(creator.memory_usage() > 0); diff --git a/src/index/src/bloom_filter/creator/finalize_segment.rs b/src/index/src/bloom_filter/creator/finalize_segment.rs new file mode 100644 index 000000000000..65b090de3eee --- /dev/null +++ b/src/index/src/bloom_filter/creator/finalize_segment.rs @@ -0,0 +1,293 @@ +// 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::pin::Pin; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::Arc; + +use asynchronous_codec::{FramedRead, FramedWrite}; +use fastbloom::BloomFilter; +use futures::stream::StreamExt; +use futures::{stream, AsyncWriteExt, Stream}; +use snafu::ResultExt; + +use super::intermediate_codec::IntermediateBloomFilterCodecV1; +use crate::bloom_filter::creator::{FALSE_POSITIVE_RATE, SEED}; +use crate::bloom_filter::error::{IntermediateSnafu, IoSnafu, Result}; +use crate::bloom_filter::Bytes; +use crate::external_provider::ExternalTempFileProvider; + +/// The minimum memory usage threshold for flushing in-memory Bloom filters to disk. +const MIN_MEMORY_USAGE_THRESHOLD: usize = 1024 * 1024; // 1MB + +/// Storage for finalized Bloom filters. +pub struct FinalizedBloomFilterStorage { + /// Bloom filters that are stored in memory. + in_memory: Vec, + + /// Used to generate unique file IDs for intermediate Bloom filters. + intermediate_file_id_counter: usize, + + /// Prefix for intermediate Bloom filter files. + intermediate_prefix: String, + + /// The provider for intermediate Bloom filter files. + intermediate_provider: Box, + + /// The memory usage of the in-memory Bloom filters. + memory_usage: usize, + + /// The global memory usage provided by the user to track the + /// total memory usage of the creating Bloom filters. + global_memory_usage: Arc, + + /// The threshold of the global memory usage of the creating Bloom filters. + global_memory_usage_threshold: Option, +} + +impl FinalizedBloomFilterStorage { + /// Creates a new `FinalizedBloomFilterStorage`. + pub fn new( + intermediate_provider: Box, + global_memory_usage: Arc, + global_memory_usage_threshold: Option, + ) -> Self { + let external_prefix = format!("intm-bloom-filters-{}", uuid::Uuid::new_v4()); + Self { + in_memory: Vec::new(), + intermediate_file_id_counter: 0, + intermediate_prefix: external_prefix, + intermediate_provider, + memory_usage: 0, + global_memory_usage, + global_memory_usage_threshold, + } + } + + /// Returns the memory usage of the storage. + pub fn memory_usage(&self) -> usize { + self.memory_usage + } + + /// Adds a new finalized Bloom filter to the storage. + /// + /// If the memory usage exceeds the threshold, flushes the in-memory Bloom filters to disk. + pub async fn add( + &mut self, + elems: impl IntoIterator, + element_count: usize, + ) -> Result<()> { + let mut bf = BloomFilter::with_false_pos(FALSE_POSITIVE_RATE) + .seed(&SEED) + .expected_items(element_count); + for elem in elems.into_iter() { + bf.insert(&elem); + } + + let fbf = FinalizedBloomFilterSegment::from(bf, element_count); + + // Update memory usage. + let memory_diff = fbf.bloom_filter_bytes.len(); + self.memory_usage += memory_diff; + self.global_memory_usage + .fetch_add(memory_diff, Ordering::Relaxed); + + // Add the finalized Bloom filter to the in-memory storage. + self.in_memory.push(fbf); + + // Flush to disk if necessary. + + // Do not flush if memory usage is too low. + if self.memory_usage < MIN_MEMORY_USAGE_THRESHOLD { + return Ok(()); + } + + // Check if the global memory usage exceeds the threshold and flush to disk if necessary. + if let Some(threshold) = self.global_memory_usage_threshold { + let global = self.global_memory_usage.load(Ordering::Relaxed); + + if global > threshold { + self.flush_in_memory_to_disk().await?; + + self.global_memory_usage + .fetch_sub(self.memory_usage, Ordering::Relaxed); + self.memory_usage = 0; + } + } + + Ok(()) + } + + /// Drains the storage and returns a stream of finalized Bloom filter segments. + pub async fn drain( + &mut self, + ) -> Result> + '_>>> { + // FAST PATH: memory only + if self.intermediate_file_id_counter == 0 { + return Ok(Box::pin(stream::iter(self.in_memory.drain(..).map(Ok)))); + } + + // SLOW PATH: memory + disk + let mut on_disk = self + .intermediate_provider + .read_all(&self.intermediate_prefix) + .await + .context(IntermediateSnafu)?; + on_disk.sort_unstable_by(|x, y| x.0.cmp(&y.0)); + + let streams = on_disk + .into_iter() + .map(|(_, reader)| FramedRead::new(reader, IntermediateBloomFilterCodecV1::default())); + + let in_memory_stream = stream::iter(self.in_memory.drain(..)).map(Ok); + Ok(Box::pin( + stream::iter(streams).flatten().chain(in_memory_stream), + )) + } + + /// Flushes the in-memory Bloom filters to disk. + async fn flush_in_memory_to_disk(&mut self) -> Result<()> { + let file_id = self.intermediate_file_id_counter; + self.intermediate_file_id_counter += 1; + + let file_id = format!("{:08}", file_id); + let mut writer = self + .intermediate_provider + .create(&self.intermediate_prefix, &file_id) + .await + .context(IntermediateSnafu)?; + + let fw = FramedWrite::new(&mut writer, IntermediateBloomFilterCodecV1::default()); + // `forward()` will flush and close the writer when the stream ends + if let Err(e) = stream::iter(self.in_memory.drain(..).map(Ok)) + .forward(fw) + .await + { + writer.close().await.context(IoSnafu)?; + writer.flush().await.context(IoSnafu)?; + return Err(e); + } + + Ok(()) + } +} + +/// A finalized Bloom filter segment. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct FinalizedBloomFilterSegment { + /// The underlying Bloom filter bytes. + pub bloom_filter_bytes: Vec, + + /// The number of elements in the Bloom filter. + pub element_count: usize, +} + +impl FinalizedBloomFilterSegment { + fn from(bf: BloomFilter, elem_count: usize) -> Self { + let bf_slice = bf.as_slice(); + let mut bloom_filter_bytes = Vec::with_capacity(std::mem::size_of_val(bf_slice)); + for &x in bf_slice { + bloom_filter_bytes.extend_from_slice(&x.to_le_bytes()); + } + + Self { + bloom_filter_bytes, + element_count: elem_count, + } + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + use std::sync::Mutex; + + use futures::AsyncRead; + use tokio::io::duplex; + use tokio_util::compat::{TokioAsyncReadCompatExt, TokioAsyncWriteCompatExt}; + + use super::*; + use crate::bloom_filter::creator::tests::u64_vec_from_bytes; + use crate::external_provider::MockExternalTempFileProvider; + + #[tokio::test] + async fn test_finalized_bloom_filter_storage() { + let mut mock_provider = MockExternalTempFileProvider::new(); + + let mock_files: Arc>>> = + Arc::new(Mutex::new(HashMap::new())); + + mock_provider.expect_create().returning({ + let files = Arc::clone(&mock_files); + move |file_group, file_id| { + assert!(file_group.starts_with("intm-bloom-filters-")); + let mut files = files.lock().unwrap(); + let (writer, reader) = duplex(2 * 1024 * 1024); + files.insert(file_id.to_string(), Box::new(reader.compat())); + Ok(Box::new(writer.compat_write())) + } + }); + + mock_provider.expect_read_all().returning({ + let files = Arc::clone(&mock_files); + move |file_group| { + assert!(file_group.starts_with("intm-bloom-filters-")); + let mut files = files.lock().unwrap(); + Ok(files.drain().collect::>()) + } + }); + + let global_memory_usage = Arc::new(AtomicUsize::new(0)); + let global_memory_usage_threshold = Some(1024 * 1024); // 1MB + let provider = Box::new(mock_provider); + let mut storage = FinalizedBloomFilterStorage::new( + provider, + global_memory_usage.clone(), + global_memory_usage_threshold, + ); + + let elem_count = 2000; + let batch = 1000; + + for i in 0..batch { + let elems = (elem_count * i..elem_count * (i + 1)).map(|x| x.to_string().into_bytes()); + storage.add(elems, elem_count).await.unwrap(); + } + + // Flush happens. + assert!(storage.intermediate_file_id_counter > 0); + + // Drain the storage. + let mut stream = storage.drain().await.unwrap(); + + let mut i = 0; + while let Some(segment) = stream.next().await { + let segment = segment.unwrap(); + assert_eq!(segment.element_count, elem_count); + + let v = u64_vec_from_bytes(&segment.bloom_filter_bytes); + + // Check the correctness of the Bloom filter. + let bf = BloomFilter::from_vec(v) + .seed(&SEED) + .expected_items(segment.element_count); + for elem in (elem_count * i..elem_count * (i + 1)).map(|x| x.to_string().into_bytes()) { + assert!(bf.contains(&elem)); + } + i += 1; + } + + assert_eq!(i, batch); + } +} diff --git a/src/index/src/bloom_filter/creator/intermediate_codec.rs b/src/index/src/bloom_filter/creator/intermediate_codec.rs new file mode 100644 index 000000000000..a01d7d72510d --- /dev/null +++ b/src/index/src/bloom_filter/creator/intermediate_codec.rs @@ -0,0 +1,248 @@ +// 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 asynchronous_codec::{BytesMut, Decoder, Encoder}; +use bytes::{Buf, BufMut}; +use snafu::{ensure, ResultExt}; + +use crate::bloom_filter::creator::finalize_segment::FinalizedBloomFilterSegment; +use crate::bloom_filter::error::{Error, InvalidIntermediateMagicSnafu, IoSnafu, Result}; + +/// The magic number for the codec version 1 of the intermediate bloom filter. +const CODEC_V1_MAGIC: &[u8; 4] = b"bi01"; + +/// Codec of the intermediate finalized bloom filter segment. +/// +/// # Format +/// +/// [ magic ][ elem count ][ size ][ bloom filter ][ elem count ][ size ][ bloom filter ]... +/// [4] [8] [8] [size] [8] [8] [size] +#[derive(Debug, Default)] +pub struct IntermediateBloomFilterCodecV1 { + handled_header_magic: bool, +} + +impl Encoder for IntermediateBloomFilterCodecV1 { + type Item<'a> = FinalizedBloomFilterSegment; + type Error = Error; + + fn encode(&mut self, item: FinalizedBloomFilterSegment, dst: &mut BytesMut) -> Result<()> { + if !self.handled_header_magic { + dst.extend_from_slice(CODEC_V1_MAGIC); + self.handled_header_magic = true; + } + + let segment_bytes = item.bloom_filter_bytes; + let elem_count = item.element_count; + + dst.reserve(2 * std::mem::size_of::() + segment_bytes.len()); + dst.put_u64_le(elem_count as u64); + dst.put_u64_le(segment_bytes.len() as u64); + dst.extend_from_slice(&segment_bytes); + Ok(()) + } +} + +impl Decoder for IntermediateBloomFilterCodecV1 { + type Item = FinalizedBloomFilterSegment; + type Error = Error; + + fn decode(&mut self, src: &mut BytesMut) -> Result> { + if !self.handled_header_magic { + let m_len = CODEC_V1_MAGIC.len(); + if src.remaining() < m_len { + return Ok(None); + } + let magic_bytes = &src[..m_len]; + ensure!( + magic_bytes == CODEC_V1_MAGIC, + InvalidIntermediateMagicSnafu { + invalid: magic_bytes, + } + ); + self.handled_header_magic = true; + src.advance(m_len); + } + + let s = &src[..]; + + let u64_size = std::mem::size_of::(); + let n_size = u64_size * 2; + if s.len() < n_size { + return Ok(None); + } + + let element_count = u64::from_le_bytes(s[0..u64_size].try_into().unwrap()) as usize; + let segment_size = u64::from_le_bytes(s[u64_size..n_size].try_into().unwrap()) as usize; + + if s.len() < n_size + segment_size { + return Ok(None); + } + + let bloom_filter_bytes = s[n_size..n_size + segment_size].to_vec(); + src.advance(n_size + segment_size); + Ok(Some(FinalizedBloomFilterSegment { + element_count, + bloom_filter_bytes, + })) + } +} + +/// Required for [`Encoder`] and [`Decoder`] implementations. +impl From for Error { + fn from(error: std::io::Error) -> Self { + Err::<(), std::io::Error>(error) + .context(IoSnafu) + .unwrap_err() + } +} + +#[cfg(test)] +mod tests { + use asynchronous_codec::{FramedRead, FramedWrite}; + use futures::io::Cursor; + use futures::{SinkExt, StreamExt}; + + use super::*; + use crate::bloom_filter::creator::finalize_segment::FinalizedBloomFilterSegment; + + #[test] + fn test_intermediate_bloom_filter_codec_v1_basic() { + let mut encoder = IntermediateBloomFilterCodecV1::default(); + let mut buf = BytesMut::new(); + + let item1 = FinalizedBloomFilterSegment { + element_count: 2, + bloom_filter_bytes: vec![1, 2, 3, 4], + }; + let item2 = FinalizedBloomFilterSegment { + element_count: 3, + bloom_filter_bytes: vec![5, 6, 7, 8], + }; + let item3 = FinalizedBloomFilterSegment { + element_count: 4, + bloom_filter_bytes: vec![9, 10, 11, 12], + }; + + encoder.encode(item1.clone(), &mut buf).unwrap(); + encoder.encode(item2.clone(), &mut buf).unwrap(); + encoder.encode(item3.clone(), &mut buf).unwrap(); + + let mut buf = buf.freeze().try_into_mut().unwrap(); + + let mut decoder = IntermediateBloomFilterCodecV1::default(); + let decoded_item1 = decoder.decode(&mut buf).unwrap().unwrap(); + let decoded_item2 = decoder.decode(&mut buf).unwrap().unwrap(); + let decoded_item3 = decoder.decode(&mut buf).unwrap().unwrap(); + + assert_eq!(item1, decoded_item1); + assert_eq!(item2, decoded_item2); + assert_eq!(item3, decoded_item3); + } + + #[tokio::test] + async fn test_intermediate_bloom_filter_codec_v1_frame_read_write() { + let item1 = FinalizedBloomFilterSegment { + element_count: 2, + bloom_filter_bytes: vec![1, 2, 3, 4], + }; + let item2 = FinalizedBloomFilterSegment { + element_count: 3, + bloom_filter_bytes: vec![5, 6, 7, 8], + }; + let item3 = FinalizedBloomFilterSegment { + element_count: 4, + bloom_filter_bytes: vec![9, 10, 11, 12], + }; + + let mut bytes = Cursor::new(vec![]); + + let mut writer = FramedWrite::new(&mut bytes, IntermediateBloomFilterCodecV1::default()); + writer.send(item1.clone()).await.unwrap(); + writer.send(item2.clone()).await.unwrap(); + writer.send(item3.clone()).await.unwrap(); + writer.flush().await.unwrap(); + writer.close().await.unwrap(); + + let bytes = bytes.into_inner(); + let mut reader = + FramedRead::new(bytes.as_slice(), IntermediateBloomFilterCodecV1::default()); + let decoded_item1 = reader.next().await.unwrap().unwrap(); + let decoded_item2 = reader.next().await.unwrap().unwrap(); + let decoded_item3 = reader.next().await.unwrap().unwrap(); + assert!(reader.next().await.is_none()); + + assert_eq!(item1, decoded_item1); + assert_eq!(item2, decoded_item2); + assert_eq!(item3, decoded_item3); + } + + #[tokio::test] + async fn test_intermediate_bloom_filter_codec_v1_frame_read_write_only_magic() { + let bytes = CODEC_V1_MAGIC.to_vec(); + let mut reader = + FramedRead::new(bytes.as_slice(), IntermediateBloomFilterCodecV1::default()); + assert!(reader.next().await.is_none()); + } + + #[tokio::test] + async fn test_intermediate_bloom_filter_codec_v1_frame_read_write_partial_magic() { + let bytes = CODEC_V1_MAGIC[..3].to_vec(); + let mut reader = + FramedRead::new(bytes.as_slice(), IntermediateBloomFilterCodecV1::default()); + let e = reader.next().await.unwrap(); + assert!(e.is_err()); + } + + #[tokio::test] + async fn test_intermediate_bloom_filter_codec_v1_frame_read_write_partial_item() { + let mut bytes = vec![]; + bytes.extend_from_slice(CODEC_V1_MAGIC); + bytes.extend_from_slice(&2u64.to_le_bytes()); + bytes.extend_from_slice(&4u64.to_le_bytes()); + + let mut reader = + FramedRead::new(bytes.as_slice(), IntermediateBloomFilterCodecV1::default()); + let e = reader.next().await.unwrap(); + assert!(e.is_err()); + } + + #[tokio::test] + async fn test_intermediate_bloom_filter_codec_v1_frame_read_write_corrupted_magic() { + let mut bytes = vec![]; + bytes.extend_from_slice(b"bi02"); + bytes.extend_from_slice(&2u64.to_le_bytes()); + bytes.extend_from_slice(&4u64.to_le_bytes()); + bytes.extend_from_slice(&[1, 2, 3, 4]); + + let mut reader = + FramedRead::new(bytes.as_slice(), IntermediateBloomFilterCodecV1::default()); + let e = reader.next().await.unwrap(); + assert!(e.is_err()); + } + + #[tokio::test] + async fn test_intermediate_bloom_filter_codec_v1_frame_read_write_corrupted_length() { + let mut bytes = vec![]; + bytes.extend_from_slice(CODEC_V1_MAGIC); + bytes.extend_from_slice(&2u64.to_le_bytes()); + bytes.extend_from_slice(&4u64.to_le_bytes()); + bytes.extend_from_slice(&[1, 2, 3]); + + let mut reader = + FramedRead::new(bytes.as_slice(), IntermediateBloomFilterCodecV1::default()); + let e = reader.next().await.unwrap(); + assert!(e.is_err()); + } +} diff --git a/src/index/src/bloom_filter/error.rs b/src/index/src/bloom_filter/error.rs index 8e95dc52255e..7b91061aea05 100644 --- a/src/index/src/bloom_filter/error.rs +++ b/src/index/src/bloom_filter/error.rs @@ -39,6 +39,20 @@ pub enum Error { location: Location, }, + #[snafu(display("Intermediate error"))] + Intermediate { + source: crate::error::Error, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Invalid intermediate magic"))] + InvalidIntermediateMagic { + invalid: Vec, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("External error"))] External { source: BoxedError, @@ -52,8 +66,11 @@ impl ErrorExt for Error { use Error::*; match self { - Io { .. } | Self::SerdeJson { .. } => StatusCode::Unexpected, + Io { .. } | SerdeJson { .. } | InvalidIntermediateMagic { .. } => { + StatusCode::Unexpected + } + Intermediate { source, .. } => source.status_code(), External { source, .. } => source.status_code(), } } diff --git a/src/index/src/error.rs b/src/index/src/error.rs new file mode 100644 index 000000000000..7214437bbbe4 --- /dev/null +++ b/src/index/src/error.rs @@ -0,0 +1,48 @@ +// 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::any::Any; + +use common_error::ext::{BoxedError, ErrorExt}; +use common_error::status_code::StatusCode; +use common_macro::stack_trace_debug; +use snafu::{Location, Snafu}; + +#[derive(Snafu)] +#[snafu(visibility(pub))] +#[stack_trace_debug] +pub enum Error { + #[snafu(display("External error"))] + External { + source: BoxedError, + #[snafu(implicit)] + location: Location, + }, +} + +impl ErrorExt for Error { + fn status_code(&self) -> StatusCode { + use Error::*; + + match self { + External { source, .. } => source.status_code(), + } + } + + fn as_any(&self) -> &dyn Any { + self + } +} + +pub type Result = std::result::Result; diff --git a/src/index/src/inverted_index/create/sort/external_provider.rs b/src/index/src/external_provider.rs similarity index 58% rename from src/index/src/inverted_index/create/sort/external_provider.rs rename to src/index/src/external_provider.rs index a86f3e06aad4..60d2fac2eaec 100644 --- a/src/index/src/inverted_index/create/sort/external_provider.rs +++ b/src/index/src/external_provider.rs @@ -15,25 +15,24 @@ use async_trait::async_trait; use futures::{AsyncRead, AsyncWrite}; -use crate::inverted_index::error::Result; +use crate::error::Error; -/// Trait for managing intermediate files during external sorting for a particular index. +pub type Writer = Box; +pub type Reader = Box; + +/// Trait for managing intermediate files to control memory usage for a particular index. #[mockall::automock] #[async_trait] pub trait ExternalTempFileProvider: Send + Sync { - /// Creates and opens a new intermediate file associated with a specific index for writing. + /// Creates and opens a new intermediate file associated with a specific `file_group` for writing. /// The implementation should ensure that the file does not already exist. /// - /// - `index_name`: the name of the index for which the file will be associated + /// - `file_group`: a unique identifier for the group of files /// - `file_id`: a unique identifier for the new file - async fn create( - &self, - index_name: &str, - file_id: &str, - ) -> Result>; + async fn create(&self, file_group: &str, file_id: &str) -> Result; - /// Retrieves all intermediate files associated with a specific index for an external sorting operation. + /// Retrieves all intermediate files and their associated file identifiers for a specific `file_group`. /// - /// `index_name`: the name of the index to retrieve intermediate files for - async fn read_all(&self, index_name: &str) -> Result>>; + /// `file_group` is a unique identifier for the group of files. + async fn read_all(&self, file_group: &str) -> Result, Error>; } diff --git a/src/index/src/inverted_index/create/sort.rs b/src/index/src/inverted_index/create/sort.rs index 369017835643..81ca9aeca690 100644 --- a/src/index/src/inverted_index/create/sort.rs +++ b/src/index/src/inverted_index/create/sort.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod external_provider; pub mod external_sort; mod intermediate_rw; mod merge_stream; diff --git a/src/index/src/inverted_index/create/sort/external_sort.rs b/src/index/src/inverted_index/create/sort/external_sort.rs index 17afd7ced31c..f4e1d9f9101d 100644 --- a/src/index/src/inverted_index/create/sort/external_sort.rs +++ b/src/index/src/inverted_index/create/sort/external_sort.rs @@ -23,15 +23,16 @@ use async_trait::async_trait; use common_base::BitVec; use common_telemetry::{debug, error}; use futures::stream; +use snafu::ResultExt; -use crate::inverted_index::create::sort::external_provider::ExternalTempFileProvider; +use crate::external_provider::ExternalTempFileProvider; use crate::inverted_index::create::sort::intermediate_rw::{ IntermediateReader, IntermediateWriter, }; use crate::inverted_index::create::sort::merge_stream::MergeSortedStream; use crate::inverted_index::create::sort::{SortOutput, SortedStream, Sorter}; use crate::inverted_index::create::sort_create::SorterFactory; -use crate::inverted_index::error::Result; +use crate::inverted_index::error::{IntermediateSnafu, Result}; use crate::inverted_index::{Bytes, BytesRef}; /// `ExternalSorter` manages the sorting of data using both in-memory structures and external files. @@ -107,7 +108,11 @@ impl Sorter for ExternalSorter { /// Finalizes the sorting operation, merging data from both in-memory buffer and external files /// into a sorted stream async fn output(&mut self) -> Result { - let readers = self.temp_file_provider.read_all(&self.index_name).await?; + let readers = self + .temp_file_provider + .read_all(&self.index_name) + .await + .context(IntermediateSnafu)?; // TODO(zhongzc): k-way merge instead of 2-way merge @@ -122,7 +127,7 @@ impl Sorter for ExternalSorter { Ok((value, bitmap)) }), ))); - for reader in readers { + for (_, reader) in readers { tree_nodes.push_back(IntermediateReader::new(reader).into_stream().await?); } @@ -241,7 +246,11 @@ impl ExternalSorter { let file_id = &format!("{:012}", self.total_row_count); let index_name = &self.index_name; - let writer = self.temp_file_provider.create(index_name, file_id).await?; + let writer = self + .temp_file_provider + .create(index_name, file_id) + .await + .context(IntermediateSnafu)?; let values = mem::take(&mut self.values_buffer); self.global_memory_usage @@ -302,7 +311,7 @@ mod tests { use tokio_util::compat::{TokioAsyncReadCompatExt, TokioAsyncWriteCompatExt}; use super::*; - use crate::inverted_index::create::sort::external_provider::MockExternalTempFileProvider; + use crate::external_provider::MockExternalTempFileProvider; async fn test_external_sorter( current_memory_usage_threshold: Option, @@ -332,7 +341,7 @@ mod tests { move |index_name| { assert_eq!(index_name, "test"); let mut files = files.lock().unwrap(); - Ok(files.drain().map(|f| f.1).collect::>()) + Ok(files.drain().collect::>()) } }); diff --git a/src/index/src/inverted_index/error.rs b/src/index/src/inverted_index/error.rs index 7e861beda6d1..c53e2ae9f57e 100644 --- a/src/index/src/inverted_index/error.rs +++ b/src/index/src/inverted_index/error.rs @@ -213,6 +213,13 @@ pub enum Error { #[snafu(implicit)] location: Location, }, + + #[snafu(display("Intermediate error"))] + Intermediate { + source: crate::error::Error, + #[snafu(implicit)] + location: Location, + }, } impl ErrorExt for Error { @@ -245,6 +252,7 @@ impl ErrorExt for Error { | InconsistentRowCount { .. } | IndexNotFound { .. } => StatusCode::InvalidArguments, + Intermediate { source, .. } => source.status_code(), External { source, .. } => source.status_code(), } } diff --git a/src/index/src/lib.rs b/src/index/src/lib.rs index e52a93138f68..e490dbc06464 100644 --- a/src/index/src/lib.rs +++ b/src/index/src/lib.rs @@ -16,5 +16,7 @@ #![feature(assert_matches)] pub mod bloom_filter; +pub mod error; +pub mod external_provider; pub mod fulltext_index; pub mod inverted_index; diff --git a/src/mito2/src/sst/index/intermediate.rs b/src/mito2/src/sst/index/intermediate.rs index 1568261e206f..d0da804c745b 100644 --- a/src/mito2/src/sst/index/intermediate.rs +++ b/src/mito2/src/sst/index/intermediate.rs @@ -104,16 +104,28 @@ impl IntermediateLocation { &self.files_dir } - /// Returns the path of the directory for intermediate files associated with a column: - /// `__intm/{region_id}/{sst_file_id}/{uuid}/{column_id}/` - pub fn column_path(&self, column_id: &str) -> String { - util::join_path(&self.files_dir, &format!("{column_id}/")) + /// Returns the path of the directory for intermediate files associated with the `file_group`: + /// `__intm/{region_id}/{sst_file_id}/{uuid}/{file_group}/` + pub fn file_group_path(&self, file_group: &str) -> String { + util::join_path(&self.files_dir, &format!("{file_group}/")) } - /// Returns the path of the intermediate file with the given id for a column: - /// `__intm/{region_id}/{sst_file_id}/{uuid}/{column_id}/{im_file_id}.im` - pub fn file_path(&self, column_id: &str, im_file_id: &str) -> String { - util::join_path(&self.column_path(column_id), &format!("{im_file_id}.im")) + /// Returns the path of the intermediate file with the given `file_group` and `im_file_id`: + /// `__intm/{region_id}/{sst_file_id}/{uuid}/{file_group}/{im_file_id}.im` + pub fn file_path(&self, file_group: &str, im_file_id: &str) -> String { + util::join_path( + &self.file_group_path(file_group), + &format!("{im_file_id}.im"), + ) + } + + /// Returns the intermediate file id from the path. + pub fn im_file_id_from_path(&self, path: &str) -> String { + path.rsplit('/') + .next() + .and_then(|s| s.strip_suffix(".im")) + .unwrap_or_default() + .to_string() } } @@ -161,17 +173,20 @@ mod tests { let uuid = location.files_dir.split('/').nth(3).unwrap(); - let column_id = "1"; + let file_group = "1"; assert_eq!( - location.column_path(column_id), - format!("{INTERMEDIATE_DIR}/0/{sst_file_id}/{uuid}/{column_id}/") + location.file_group_path(file_group), + format!("{INTERMEDIATE_DIR}/0/{sst_file_id}/{uuid}/{file_group}/") ); let im_file_id = "000000000010"; + let file_path = location.file_path(file_group, im_file_id); assert_eq!( - location.file_path(column_id, im_file_id), - format!("{INTERMEDIATE_DIR}/0/{sst_file_id}/{uuid}/{column_id}/{im_file_id}.im") + file_path, + format!("{INTERMEDIATE_DIR}/0/{sst_file_id}/{uuid}/{file_group}/{im_file_id}.im") ); + + assert_eq!(location.im_file_id_from_path(&file_path), im_file_id); } #[tokio::test] diff --git a/src/mito2/src/sst/index/inverted_index/creator/temp_provider.rs b/src/mito2/src/sst/index/inverted_index/creator/temp_provider.rs index ee80aaa0a63c..1822f3119459 100644 --- a/src/mito2/src/sst/index/inverted_index/creator/temp_provider.rs +++ b/src/mito2/src/sst/index/inverted_index/creator/temp_provider.rs @@ -16,9 +16,9 @@ use async_trait::async_trait; use common_error::ext::BoxedError; use common_telemetry::warn; use futures::{AsyncRead, AsyncWrite}; -use index::inverted_index::create::sort::external_provider::ExternalTempFileProvider; -use index::inverted_index::error as index_error; -use index::inverted_index::error::Result as IndexResult; +use index::error as index_error; +use index::error::Result as IndexResult; +use index::external_provider::ExternalTempFileProvider; use snafu::ResultExt; use crate::error::Result; @@ -42,10 +42,10 @@ pub(crate) struct TempFileProvider { impl ExternalTempFileProvider for TempFileProvider { async fn create( &self, - column_id: &str, + file_group: &str, file_id: &str, ) -> IndexResult> { - let path = self.location.file_path(column_id, file_id); + let path = self.location.file_path(file_group, file_id); let writer = self .manager .store() @@ -63,13 +63,13 @@ impl ExternalTempFileProvider for TempFileProvider { async fn read_all( &self, - column_id: &str, - ) -> IndexResult>> { - let column_path = self.location.column_path(column_id); + file_group: &str, + ) -> IndexResult)>> { + let file_group_path = self.location.file_group_path(file_group); let entries = self .manager .store() - .list(&column_path) + .list(&file_group_path) .await .map_err(BoxedError::new) .context(index_error::ExternalSnafu)?; @@ -81,6 +81,8 @@ impl ExternalTempFileProvider for TempFileProvider { continue; } + let im_file_id = self.location.im_file_id_from_path(entry.path()); + let reader = self .manager .store() @@ -93,7 +95,7 @@ impl ExternalTempFileProvider for TempFileProvider { .await .map_err(BoxedError::new) .context(index_error::ExternalSnafu)?; - readers.push(Box::new(reader) as _); + readers.push((im_file_id, Box::new(reader) as _)); } Ok(readers) @@ -133,36 +135,36 @@ mod tests { let store = IntermediateManager::init_fs(path).await.unwrap(); let provider = TempFileProvider::new(location.clone(), store); - let column_name = "tag0"; + let file_group = "tag0"; let file_id = "0000000010"; - let mut writer = provider.create(column_name, file_id).await.unwrap(); + let mut writer = provider.create(file_group, file_id).await.unwrap(); writer.write_all(b"hello").await.unwrap(); writer.flush().await.unwrap(); writer.close().await.unwrap(); let file_id = "0000000100"; - let mut writer = provider.create(column_name, file_id).await.unwrap(); + let mut writer = provider.create(file_group, file_id).await.unwrap(); writer.write_all(b"world").await.unwrap(); writer.flush().await.unwrap(); writer.close().await.unwrap(); - let column_name = "tag1"; + let file_group = "tag1"; let file_id = "0000000010"; - let mut writer = provider.create(column_name, file_id).await.unwrap(); + let mut writer = provider.create(file_group, file_id).await.unwrap(); writer.write_all(b"foo").await.unwrap(); writer.flush().await.unwrap(); writer.close().await.unwrap(); let readers = provider.read_all("tag0").await.unwrap(); assert_eq!(readers.len(), 2); - for mut reader in readers { + for (_, mut reader) in readers { let mut buf = Vec::new(); reader.read_to_end(&mut buf).await.unwrap(); assert!(matches!(buf.as_slice(), b"hello" | b"world")); } let readers = provider.read_all("tag1").await.unwrap(); assert_eq!(readers.len(), 1); - let mut reader = readers.into_iter().next().unwrap(); + let mut reader = readers.into_iter().map(|x| x.1).next().unwrap(); let mut buf = Vec::new(); reader.read_to_end(&mut buf).await.unwrap(); assert_eq!(buf, b"foo"); From 4245bff8f208ed801f3f57c40371aa19c2a57883 Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Fri, 20 Dec 2024 16:29:18 +0800 Subject: [PATCH 08/12] feat(bloom-filter): add bloom filter reader (#5204) * feat(bloom-filter): add bloom filter reader Signed-off-by: Zhenchi * chore: remove unused dep Signed-off-by: Zhenchi * fix conflict Signed-off-by: Zhenchi * address comments Signed-off-by: Zhenchi --------- Signed-off-by: Zhenchi --- Cargo.lock | 1 - src/index/Cargo.toml | 1 - src/index/src/bloom_filter.rs | 6 +- src/index/src/bloom_filter/creator.rs | 8 +- src/index/src/bloom_filter/error.rs | 32 +++- src/index/src/bloom_filter/reader.rs | 265 ++++++++++++++++++++++++++ 6 files changed, 301 insertions(+), 12 deletions(-) create mode 100644 src/index/src/bloom_filter/reader.rs diff --git a/Cargo.lock b/Cargo.lock index 8f5b1037193b..c23acf60636d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5270,7 +5270,6 @@ dependencies = [ name = "index" version = "0.12.0" dependencies = [ - "async-stream", "async-trait", "asynchronous-codec", "bytemuck", diff --git a/src/index/Cargo.toml b/src/index/Cargo.toml index f91a8eac9708..f46c64a17606 100644 --- a/src/index/Cargo.toml +++ b/src/index/Cargo.toml @@ -8,7 +8,6 @@ license.workspace = true workspace = true [dependencies] -async-stream.workspace = true async-trait.workspace = true asynchronous-codec = "0.7.0" bytemuck.workspace = true diff --git a/src/index/src/bloom_filter.rs b/src/index/src/bloom_filter.rs index e68acc698a26..00e80f7490ee 100644 --- a/src/index/src/bloom_filter.rs +++ b/src/index/src/bloom_filter.rs @@ -15,11 +15,15 @@ use serde::{Deserialize, Serialize}; pub mod creator; -mod error; +pub mod error; +pub mod reader; pub type Bytes = Vec; pub type BytesRef<'a> = &'a [u8]; +/// The seed used for the Bloom filter. +pub const SEED: u128 = 42; + /// The Meta information of the bloom filter stored in the file. #[derive(Debug, Default, Serialize, Deserialize)] pub struct BloomFilterMeta { diff --git a/src/index/src/bloom_filter/creator.rs b/src/index/src/bloom_filter/creator.rs index 04d2edc3d1b3..2f10521559a5 100644 --- a/src/index/src/bloom_filter/creator.rs +++ b/src/index/src/bloom_filter/creator.rs @@ -23,14 +23,10 @@ use finalize_segment::FinalizedBloomFilterStorage; use futures::{AsyncWrite, AsyncWriteExt, StreamExt}; use snafu::ResultExt; -use super::error::{IoSnafu, SerdeJsonSnafu}; -use crate::bloom_filter::error::Result; -use crate::bloom_filter::{BloomFilterMeta, BloomFilterSegmentLocation, Bytes}; +use crate::bloom_filter::error::{IoSnafu, Result, SerdeJsonSnafu}; +use crate::bloom_filter::{BloomFilterMeta, BloomFilterSegmentLocation, Bytes, SEED}; use crate::external_provider::ExternalTempFileProvider; -/// The seed used for the Bloom filter. -pub const SEED: u128 = 42; - /// The false positive rate of the Bloom filter. pub const FALSE_POSITIVE_RATE: f64 = 0.01; diff --git a/src/index/src/bloom_filter/error.rs b/src/index/src/bloom_filter/error.rs index 7b91061aea05..fc89a8b68adf 100644 --- a/src/index/src/bloom_filter/error.rs +++ b/src/index/src/bloom_filter/error.rs @@ -39,6 +39,14 @@ pub enum Error { location: Location, }, + #[snafu(display("Failed to deserialize json"))] + DeserializeJson { + #[snafu(source)] + error: serde_json::Error, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Intermediate error"))] Intermediate { source: crate::error::Error, @@ -46,6 +54,21 @@ pub enum Error { location: Location, }, + #[snafu(display("File size too small for bloom filter"))] + FileSizeTooSmall { + size: u64, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Unexpected bloom filter meta size"))] + UnexpectedMetaSize { + max_meta_size: u64, + actual_meta_size: u64, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Invalid intermediate magic"))] InvalidIntermediateMagic { invalid: Vec, @@ -66,9 +89,12 @@ impl ErrorExt for Error { use Error::*; match self { - Io { .. } | SerdeJson { .. } | InvalidIntermediateMagic { .. } => { - StatusCode::Unexpected - } + Io { .. } + | SerdeJson { .. } + | FileSizeTooSmall { .. } + | UnexpectedMetaSize { .. } + | DeserializeJson { .. } + | InvalidIntermediateMagic { .. } => StatusCode::Unexpected, Intermediate { source, .. } => source.status_code(), External { source, .. } => source.status_code(), diff --git a/src/index/src/bloom_filter/reader.rs b/src/index/src/bloom_filter/reader.rs new file mode 100644 index 000000000000..788afe033124 --- /dev/null +++ b/src/index/src/bloom_filter/reader.rs @@ -0,0 +1,265 @@ +// 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::ops::Range; + +use async_trait::async_trait; +use bytes::Bytes; +use common_base::range_read::RangeReader; +use fastbloom::BloomFilter; +use snafu::{ensure, ResultExt}; + +use crate::bloom_filter::error::{ + DeserializeJsonSnafu, FileSizeTooSmallSnafu, IoSnafu, Result, UnexpectedMetaSizeSnafu, +}; +use crate::bloom_filter::{BloomFilterMeta, BloomFilterSegmentLocation, SEED}; + +/// Minimum size of the bloom filter, which is the size of the length of the bloom filter. +const BLOOM_META_LEN_SIZE: u64 = 4; + +/// Default prefetch size of bloom filter meta. +pub const DEFAULT_PREFETCH_SIZE: u64 = 1024; // 1KiB + +/// `BloomFilterReader` reads the bloom filter from the file. +#[async_trait] +pub trait BloomFilterReader { + /// Reads range of bytes from the file. + async fn range_read(&mut self, offset: u64, size: u32) -> Result; + + /// Reads bunch of ranges from the file. + async fn read_vec(&mut self, ranges: &[Range]) -> Result>; + + /// Reads the meta information of the bloom filter. + async fn metadata(&mut self) -> Result; + + /// Reads a bloom filter with the given location. + async fn bloom_filter(&mut self, loc: &BloomFilterSegmentLocation) -> Result { + let bytes = self.range_read(loc.offset, loc.size as _).await?; + let vec = bytes + .chunks_exact(std::mem::size_of::()) + .map(|chunk| u64::from_le_bytes(chunk.try_into().unwrap())) + .collect(); + let bm = BloomFilter::from_vec(vec) + .seed(&SEED) + .expected_items(loc.elem_count); + Ok(bm) + } +} + +/// `BloomFilterReaderImpl` reads the bloom filter from the file. +pub struct BloomFilterReaderImpl { + /// The underlying reader. + reader: R, +} + +impl BloomFilterReaderImpl { + /// Creates a new `BloomFilterReaderImpl` with the given reader. + pub fn new(reader: R) -> Self { + Self { reader } + } +} + +#[async_trait] +impl BloomFilterReader for BloomFilterReaderImpl { + async fn range_read(&mut self, offset: u64, size: u32) -> Result { + self.reader + .read(offset..offset + size as u64) + .await + .context(IoSnafu) + } + + async fn read_vec(&mut self, ranges: &[Range]) -> Result> { + self.reader.read_vec(ranges).await.context(IoSnafu) + } + + async fn metadata(&mut self) -> Result { + let metadata = self.reader.metadata().await.context(IoSnafu)?; + let file_size = metadata.content_length; + + let mut meta_reader = + BloomFilterMetaReader::new(&mut self.reader, file_size, Some(DEFAULT_PREFETCH_SIZE)); + meta_reader.metadata().await + } +} + +/// `BloomFilterMetaReader` reads the metadata of the bloom filter. +struct BloomFilterMetaReader { + reader: R, + file_size: u64, + prefetch_size: u64, +} + +impl BloomFilterMetaReader { + pub fn new(reader: R, file_size: u64, prefetch_size: Option) -> Self { + Self { + reader, + file_size, + prefetch_size: prefetch_size + .unwrap_or(BLOOM_META_LEN_SIZE) + .max(BLOOM_META_LEN_SIZE), + } + } + + /// Reads the metadata of the bloom filter. + /// + /// It will first prefetch some bytes from the end of the file, + /// then parse the metadata from the prefetch bytes. + pub async fn metadata(&mut self) -> Result { + ensure!( + self.file_size >= BLOOM_META_LEN_SIZE, + FileSizeTooSmallSnafu { + size: self.file_size, + } + ); + + let meta_start = self.file_size.saturating_sub(self.prefetch_size); + let suffix = self + .reader + .read(meta_start..self.file_size) + .await + .context(IoSnafu)?; + let suffix_len = suffix.len(); + let length = u32::from_le_bytes(Self::read_tailing_four_bytes(&suffix)?) as u64; + self.validate_meta_size(length)?; + + if length > suffix_len as u64 - BLOOM_META_LEN_SIZE { + let metadata_start = self.file_size - length - BLOOM_META_LEN_SIZE; + let meta = self + .reader + .read(metadata_start..self.file_size - BLOOM_META_LEN_SIZE) + .await + .context(IoSnafu)?; + serde_json::from_slice(&meta).context(DeserializeJsonSnafu) + } else { + let metadata_start = self.file_size - length - BLOOM_META_LEN_SIZE - meta_start; + let meta = &suffix[metadata_start as usize..suffix_len - BLOOM_META_LEN_SIZE as usize]; + serde_json::from_slice(meta).context(DeserializeJsonSnafu) + } + } + + fn read_tailing_four_bytes(suffix: &[u8]) -> Result<[u8; 4]> { + let suffix_len = suffix.len(); + ensure!( + suffix_len >= 4, + FileSizeTooSmallSnafu { + size: suffix_len as u64 + } + ); + let mut bytes = [0; 4]; + bytes.copy_from_slice(&suffix[suffix_len - 4..suffix_len]); + + Ok(bytes) + } + + fn validate_meta_size(&self, length: u64) -> Result<()> { + let max_meta_size = self.file_size - BLOOM_META_LEN_SIZE; + ensure!( + length <= max_meta_size, + UnexpectedMetaSizeSnafu { + max_meta_size, + actual_meta_size: length, + } + ); + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use std::sync::atomic::AtomicUsize; + use std::sync::Arc; + + use futures::io::Cursor; + + use super::*; + use crate::bloom_filter::creator::BloomFilterCreator; + use crate::external_provider::MockExternalTempFileProvider; + + async fn mock_bloom_filter_bytes() -> Vec { + let mut writer = Cursor::new(vec![]); + let mut creator = BloomFilterCreator::new( + 2, + Box::new(MockExternalTempFileProvider::new()), + Arc::new(AtomicUsize::new(0)), + None, + ); + + creator + .push_row_elems(vec![b"a".to_vec(), b"b".to_vec()]) + .await + .unwrap(); + creator + .push_row_elems(vec![b"c".to_vec(), b"d".to_vec()]) + .await + .unwrap(); + creator + .push_row_elems(vec![b"e".to_vec(), b"f".to_vec()]) + .await + .unwrap(); + + creator.finish(&mut writer).await.unwrap(); + + writer.into_inner() + } + + #[tokio::test] + async fn test_bloom_filter_meta_reader() { + let bytes = mock_bloom_filter_bytes().await; + let file_size = bytes.len() as u64; + + for prefetch in [0u64, file_size / 2, file_size, file_size + 10] { + let mut reader = + BloomFilterMetaReader::new(bytes.clone(), file_size as _, Some(prefetch)); + let meta = reader.metadata().await.unwrap(); + + assert_eq!(meta.rows_per_segment, 2); + assert_eq!(meta.seg_count, 2); + assert_eq!(meta.row_count, 3); + assert_eq!(meta.bloom_filter_segments.len(), 2); + + assert_eq!(meta.bloom_filter_segments[0].offset, 0); + assert_eq!(meta.bloom_filter_segments[0].elem_count, 4); + assert_eq!( + meta.bloom_filter_segments[1].offset, + meta.bloom_filter_segments[0].size + ); + assert_eq!(meta.bloom_filter_segments[1].elem_count, 2); + } + } + + #[tokio::test] + async fn test_bloom_filter_reader() { + let bytes = mock_bloom_filter_bytes().await; + + let mut reader = BloomFilterReaderImpl::new(bytes); + let meta = reader.metadata().await.unwrap(); + + assert_eq!(meta.bloom_filter_segments.len(), 2); + let bf = reader + .bloom_filter(&meta.bloom_filter_segments[0]) + .await + .unwrap(); + assert!(bf.contains(&b"a")); + assert!(bf.contains(&b"b")); + assert!(bf.contains(&b"c")); + assert!(bf.contains(&b"d")); + + let bf = reader + .bloom_filter(&meta.bloom_filter_segments[1]) + .await + .unwrap(); + assert!(bf.contains(&b"e")); + assert!(bf.contains(&b"f")); + } +} From 8ece8530768e35b0c3ebcfd72b9b8fca8a88bbbf Mon Sep 17 00:00:00 2001 From: Yiran Date: Fri, 20 Dec 2024 20:01:57 +0800 Subject: [PATCH 09/12] fix: dead links (#5212) --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index c6910b21db9b..f3aa3551d90f 100644 --- a/README.md +++ b/README.md @@ -70,7 +70,7 @@ Our core developers have been building time-series data platforms for years. Bas * **Unified Processing of Metrics, Logs, and Events** -GreptimeDB unifies time series data processing by treating all data - whether metrics, logs, or events - as timestamped events with context. Users can analyze this data using either [SQL](https://docs.greptime.com/user-guide/query-data/sql) or [PromQL](https://docs.greptime.com/user-guide/query-data/promql) and leverage stream processing ([Flow](https://docs.greptime.com/user-guide/continuous-aggregation/overview)) to enable continuous aggregation. [Read more](https://docs.greptime.com/user-guide/concepts/data-model). +GreptimeDB unifies time series data processing by treating all data - whether metrics, logs, or events - as timestamped events with context. Users can analyze this data using either [SQL](https://docs.greptime.com/user-guide/query-data/sql) or [PromQL](https://docs.greptime.com/user-guide/query-data/promql) and leverage stream processing ([Flow](https://docs.greptime.com/user-guide/flow-computation/overview)) to enable continuous aggregation. [Read more](https://docs.greptime.com/user-guide/concepts/data-model). * **Cloud-native Distributed Database** @@ -173,7 +173,7 @@ Our official Grafana dashboard for monitoring GreptimeDB is available at [grafan ## Project Status -GreptimeDB is currently in Beta. We are targeting GA (General Availability) with v1.0 release by Early 2025. +GreptimeDB is currently in Beta. We are targeting GA (General Availability) with v1.0 release by Early 2025. While in Beta, GreptimeDB is already: From fa3b7ed5eadea3b9041551bc4d2456123288621a Mon Sep 17 00:00:00 2001 From: Yingwen Date: Sat, 21 Dec 2024 16:39:24 +0800 Subject: [PATCH 10/12] build: use 8xlarge as arm default (#5214) --- .github/workflows/release.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 3f46ef1a7bda..732423b73daa 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -31,7 +31,7 @@ on: linux_arm64_runner: type: choice description: The runner uses to build linux-arm64 artifacts - default: ec2-c6g.4xlarge-arm64 + default: ec2-c6g.8xlarge-arm64 options: - ubuntu-2204-32-cores-arm - ec2-c6g.xlarge-arm64 # 4C8G From c623404fff2463a1bfa14b93ad7ee15be422be4f Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Sat, 21 Dec 2024 18:09:32 +0800 Subject: [PATCH 11/12] ci: fix nightly ci task on nix build (#5198) --- .github/workflows/nightly-ci.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/nightly-ci.yml b/.github/workflows/nightly-ci.yml index 285fb61a7cb7..0231f82a89e6 100644 --- a/.github/workflows/nightly-ci.yml +++ b/.github/workflows/nightly-ci.yml @@ -117,7 +117,6 @@ jobs: cleanbuild-linux-nix: runs-on: ubuntu-latest-8-cores timeout-minutes: 60 - needs: [coverage, fmt, clippy, check] steps: - uses: actions/checkout@v4 - uses: cachix/install-nix-action@v27 From 2082c4b6e4b8ccb292c7ee214bf4543b98dbddd7 Mon Sep 17 00:00:00 2001 From: zyy17 Date: Tue, 24 Dec 2024 11:09:41 +0800 Subject: [PATCH 12/12] docs: add greptimedb-operator project link in 'Tools & Extensions' and other small improvements (#5216) --- README.md | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index f3aa3551d90f..bcbad56bc4f2 100644 --- a/README.md +++ b/README.md @@ -70,23 +70,23 @@ Our core developers have been building time-series data platforms for years. Bas * **Unified Processing of Metrics, Logs, and Events** -GreptimeDB unifies time series data processing by treating all data - whether metrics, logs, or events - as timestamped events with context. Users can analyze this data using either [SQL](https://docs.greptime.com/user-guide/query-data/sql) or [PromQL](https://docs.greptime.com/user-guide/query-data/promql) and leverage stream processing ([Flow](https://docs.greptime.com/user-guide/flow-computation/overview)) to enable continuous aggregation. [Read more](https://docs.greptime.com/user-guide/concepts/data-model). + GreptimeDB unifies time series data processing by treating all data - whether metrics, logs, or events - as timestamped events with context. Users can analyze this data using either [SQL](https://docs.greptime.com/user-guide/query-data/sql) or [PromQL](https://docs.greptime.com/user-guide/query-data/promql) and leverage stream processing ([Flow](https://docs.greptime.com/user-guide/flow-computation/overview)) to enable continuous aggregation. [Read more](https://docs.greptime.com/user-guide/concepts/data-model). * **Cloud-native Distributed Database** -Built for [Kubernetes](https://docs.greptime.com/user-guide/deployments/deploy-on-kubernetes/greptimedb-operator-management). GreptimeDB achieves seamless scalability with its [cloud-native architecture](https://docs.greptime.com/user-guide/concepts/architecture) of separated compute and storage, built on object storage (AWS S3, Azure Blob Storage, etc.) while enabling cross-cloud deployment through a unified data access layer. + Built for [Kubernetes](https://docs.greptime.com/user-guide/deployments/deploy-on-kubernetes/greptimedb-operator-management). GreptimeDB achieves seamless scalability with its [cloud-native architecture](https://docs.greptime.com/user-guide/concepts/architecture) of separated compute and storage, built on object storage (AWS S3, Azure Blob Storage, etc.) while enabling cross-cloud deployment through a unified data access layer. * **Performance and Cost-effective** -Written in pure Rust for superior performance and reliability. GreptimeDB features a distributed query engine with intelligent indexing to handle high cardinality data efficiently. Its optimized columnar storage achieves 50x cost efficiency on cloud object storage through advanced compression. [Benchmark reports](https://www.greptime.com/blogs/2024-09-09-report-summary). + Written in pure Rust for superior performance and reliability. GreptimeDB features a distributed query engine with intelligent indexing to handle high cardinality data efficiently. Its optimized columnar storage achieves 50x cost efficiency on cloud object storage through advanced compression. [Benchmark reports](https://www.greptime.com/blogs/2024-09-09-report-summary). * **Cloud-Edge Collaboration** -GreptimeDB seamlessly operates across cloud and edge (ARM/Android/Linux), providing consistent APIs and control plane for unified data management and efficient synchronization. [Learn how to run on Android](https://docs.greptime.com/user-guide/deployments/run-on-android/). + GreptimeDB seamlessly operates across cloud and edge (ARM/Android/Linux), providing consistent APIs and control plane for unified data management and efficient synchronization. [Learn how to run on Android](https://docs.greptime.com/user-guide/deployments/run-on-android/). * **Multi-protocol Ingestion, SQL & PromQL Ready** -Widely adopted database protocols and APIs, including MySQL, PostgreSQL, InfluxDB, OpenTelemetry, Loki and Prometheus, etc. Effortless Adoption & Seamless Migration. [Supported Protocols Overview](https://docs.greptime.com/user-guide/protocols/overview). + Widely adopted database protocols and APIs, including MySQL, PostgreSQL, InfluxDB, OpenTelemetry, Loki and Prometheus, etc. Effortless Adoption & Seamless Migration. [Supported Protocols Overview](https://docs.greptime.com/user-guide/protocols/overview). For more detailed info please read [Why GreptimeDB](https://docs.greptime.com/user-guide/concepts/why-greptimedb). @@ -138,7 +138,7 @@ Check the prerequisite: * [Rust toolchain](https://www.rust-lang.org/tools/install) (nightly) * [Protobuf compiler](https://grpc.io/docs/protoc-installation/) (>= 3.15) -* Python toolchain (optional): Required only if built with PyO3 backend. More detail for compiling with PyO3 can be found in its [documentation](https://pyo3.rs/v0.18.1/building_and_distribution#configuring-the-python-version). +* Python toolchain (optional): Required only if built with PyO3 backend. More details for compiling with PyO3 can be found in its [documentation](https://pyo3.rs/v0.18.1/building_and_distribution#configuring-the-python-version). Build GreptimeDB binary: @@ -154,6 +154,10 @@ cargo run -- standalone start ## Tools & Extensions +### Kubernetes + +- [GreptimeDB Operator](https://github.com/GrepTimeTeam/greptimedb-operator) + ### Dashboard - [The dashboard UI for GreptimeDB](https://github.com/GreptimeTeam/dashboard)