From 9a597ede2e482f222c112e03f85ec816d931f2b4 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 28 May 2024 14:17:41 +0800 Subject: [PATCH 01/13] feat(risedev): only pull docker image if not exists (#16956) Signed-off-by: Bugen Zhao --- src/risedevtool/src/task/docker_service.rs | 25 +++++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/src/risedevtool/src/task/docker_service.rs b/src/risedevtool/src/task/docker_service.rs index 97a83e711c3f7..58ff2b59648c0 100644 --- a/src/risedevtool/src/task/docker_service.rs +++ b/src/risedevtool/src/task/docker_service.rs @@ -72,6 +72,23 @@ where Self { config } } + /// Run `docker image inspect ` to check if the image exists locally. + /// + /// `docker run --pull=missing` does the same thing, but as we split the pull and run + /// into two commands while `pull` does not provide such an option, we need to check + /// the image existence manually. + fn check_image_exists(&self) -> bool { + Command::new("docker") + .arg("image") + .arg("inspect") + .arg(self.config.image()) + .stdout(Stdio::null()) + .stderr(Stdio::null()) + .status() + .map(|status| status.success()) + .unwrap_or(false) + } + fn docker_pull(&self) -> Command { let mut cmd = Command::new("docker"); cmd.arg("pull").arg(self.config.image()); @@ -118,9 +135,11 @@ where check_docker_installed()?; - ctx.pb - .set_message(format!("pulling image `{}`...", self.config.image())); - ctx.run_command(self.docker_pull())?; + if !self.check_image_exists() { + ctx.pb + .set_message(format!("pulling image `{}`...", self.config.image())); + ctx.run_command(self.docker_pull())?; + } ctx.pb.set_message("starting..."); ctx.run_command(ctx.tmux_run(self.docker_run()?)?)?; From 6e62b23928c53918edafc7d58afdda9c0478482d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 28 May 2024 06:45:02 +0000 Subject: [PATCH 02/13] chore(deps): Bump duration-str from 0.7.1 to 0.10.0 (#16935) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 6 +++--- src/connector/Cargo.toml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a5a5c5eb26fd0..b398285c98948 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4339,16 +4339,16 @@ checksum = "56ce8c6da7551ec6c462cbaf3bfbc75131ebbfa1c944aeaa9dab51ca1c5f0c3b" [[package]] name = "duration-str" -version = "0.7.1" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8bb6a301a95ba86fa0ebaf71d49ae4838c51f8b84cb88ed140dfb66452bb3c4" +checksum = "7c1a2e028bbf7921549873b291ddc0cfe08b673d9489da81ac28898cd5a0e6e0" dependencies = [ "chrono", - "nom", "rust_decimal", "serde", "thiserror", "time", + "winnow 0.6.8 (registry+https://github.com/rust-lang/crates.io-index)", ] [[package]] diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 1f5bf445da0fa..849f626b22d3f 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -54,7 +54,7 @@ clickhouse = { git = "https://github.com/risingwavelabs/clickhouse.rs", rev = "d ] } csv = "1.3" deltalake = { workspace = true } -duration-str = "0.7.1" +duration-str = "0.10.0" easy-ext = "1" enum-as-inner = "0.6" futures = { version = "0.3", default-features = false, features = ["alloc"] } From 04cd1b51321feb41b708610d9d6223deab6894ea Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Tue, 28 May 2024 14:50:20 +0800 Subject: [PATCH 03/13] feat: Replace the parallel unit mapping with worker slot mapping in frontend. (#16801) Signed-off-by: Shanicky Chen Co-authored-by: Shanicky Chen <> --- proto/batch_plan.proto | 6 +- proto/common.proto | 6 + proto/meta.proto | 39 +++-- .../src/executor/join/local_lookup_join.rs | 52 +++--- src/batch/src/task/task_execution.rs | 4 +- .../src/worker_manager/worker_node_manager.rs | 69 ++++---- .../common_service/src/observer_manager.rs | 13 +- .../src/hash/consistent_hash/mapping.rs | 100 +++++++++++- src/common/src/util/worker_util.rs | 19 --- .../src/vnode_mapping/vnode_placement.rs | 151 ++++++++++-------- src/ctl/src/cmd_impl/meta/serving.rs | 43 +++-- src/frontend/src/observer/observer_manager.rs | 45 +++--- .../src/optimizer/property/distribution.rs | 11 +- .../src/scheduler/distributed/query.rs | 10 +- .../src/scheduler/distributed/stage.rs | 39 ++--- src/frontend/src/scheduler/local.rs | 21 +-- src/frontend/src/scheduler/plan_fragmenter.rs | 28 ++-- src/meta/service/src/notification_service.rs | 24 +-- src/meta/service/src/serving_service.rs | 6 +- src/meta/src/controller/catalog.rs | 57 ++++++- src/meta/src/controller/fragment.rs | 48 ++++-- src/meta/src/controller/streaming_job.rs | 19 ++- src/meta/src/controller/utils.rs | 46 +++++- src/meta/src/manager/catalog/fragment.rs | 65 +++++--- src/meta/src/serving/mod.rs | 38 ++--- src/meta/src/stream/scale.rs | 13 +- src/rpc_client/src/meta_client.rs | 8 +- 27 files changed, 630 insertions(+), 350 deletions(-) diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index 6a254332fc712..8a196eecf5021 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -225,7 +225,7 @@ message TableFunctionNode { message TaskId { string query_id = 1; uint32 stage_id = 2; - uint32 task_id = 3; + uint64 task_id = 3; } // Every task will create N buffers (channels) for parent operators to fetch results from, @@ -233,7 +233,7 @@ message TaskId { message TaskOutputId { TaskId task_id = 1; // The id of output channel to fetch from - uint32 output_id = 2; + uint64 output_id = 2; } message LocalExecutePlan { @@ -270,7 +270,7 @@ message LocalLookupJoinNode { repeated uint32 inner_side_key = 4; uint32 lookup_prefix_len = 5; plan_common.StorageTableDesc inner_side_table_desc = 6; - repeated uint32 inner_side_vnode_mapping = 7; + repeated uint64 inner_side_vnode_mapping = 7; repeated int32 inner_side_column_ids = 8; repeated uint32 output_indices = 9; repeated common.WorkerNode worker_nodes = 10; diff --git a/proto/common.proto b/proto/common.proto index 4f0d56b4823a9..164150379c484 100644 --- a/proto/common.proto +++ b/proto/common.proto @@ -92,6 +92,12 @@ message ParallelUnitMapping { repeated uint32 data = 2; } +// Vnode mapping for stream fragments. Stores mapping from virtual node to (worker id, slot index). +message WorkerSlotMapping { + repeated uint32 original_indices = 1; + repeated uint64 data = 2; +} + message BatchQueryEpoch { oneof epoch { uint64 committed = 1; diff --git a/proto/meta.proto b/proto/meta.proto index 149b2940e2a27..df90d99f6e9e9 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -111,6 +111,16 @@ message FragmentParallelUnitMappings { repeated FragmentParallelUnitMapping mappings = 1; } +/// Worker slot mapping with fragment id, used for notification. +message FragmentWorkerSlotMapping { + uint32 fragment_id = 1; + common.WorkerSlotMapping mapping = 2; +} + +message FragmentWorkerSlotMappings { + repeated FragmentWorkerSlotMapping mappings = 1; +} + // TODO: remove this when dashboard refactored. message ActorLocation { common.WorkerNode node = 1; @@ -378,8 +388,10 @@ message SubscribeRequest { message MetaSnapshot { message SnapshotVersion { uint64 catalog_version = 1; - uint64 parallel_unit_mapping_version = 2; + reserved 2; + reserved "parallel_unit_mapping_version"; uint64 worker_node_version = 3; + uint64 streaming_worker_slot_mapping_version = 4; } repeated catalog.Database databases = 1; repeated catalog.Schema schemas = 2; @@ -392,16 +404,20 @@ message MetaSnapshot { repeated catalog.Connection connections = 17; repeated catalog.Subscription subscriptions = 19; repeated user.UserInfo users = 8; + reserved 9; + reserved "parallel_unit_mappings"; GetSessionParamsResponse session_params = 20; - // for streaming - repeated FragmentParallelUnitMapping parallel_unit_mappings = 9; repeated common.WorkerNode nodes = 10; hummock.HummockSnapshot hummock_snapshot = 11; hummock.HummockVersion hummock_version = 12; backup_service.MetaBackupManifestId meta_backup_manifest_id = 14; hummock.WriteLimits hummock_write_limits = 16; - // for serving - repeated FragmentParallelUnitMapping serving_parallel_unit_mappings = 18; + reserved 18; + reserved "serving_parallel_unit_mappings"; + + // for streaming + repeated FragmentWorkerSlotMapping streaming_worker_slot_mappings = 21; + repeated FragmentWorkerSlotMapping serving_worker_slot_mappings = 22; SnapshotVersion version = 13; } @@ -440,8 +456,6 @@ message SubscribeResponse { catalog.Function function = 6; user.UserInfo user = 11; SetSessionParamRequest session_param = 26; - // for streaming - FragmentParallelUnitMapping parallel_unit_mapping = 12; common.WorkerNode node = 13; hummock.HummockSnapshot hummock_snapshot = 14; hummock.HummockVersionDeltas hummock_version_deltas = 15; @@ -451,10 +465,15 @@ message SubscribeResponse { hummock.WriteLimits hummock_write_limits = 20; RelationGroup relation_group = 21; catalog.Connection connection = 22; - FragmentParallelUnitMappings serving_parallel_unit_mappings = 23; hummock.HummockVersionStats hummock_stats = 24; Recovery recovery = 25; + FragmentWorkerSlotMapping streaming_worker_slot_mapping = 27; + FragmentWorkerSlotMappings serving_worker_slot_mappings = 28; } + reserved 12; + reserved "parallel_unit_mapping"; + reserved 23; + reserved "serving_parallel_unit_mappings"; } service NotificationService { @@ -629,8 +648,10 @@ service SessionParamService { message GetServingVnodeMappingsRequest {} message GetServingVnodeMappingsResponse { - repeated FragmentParallelUnitMapping mappings = 1; + reserved 1; + reserved "mappings"; map fragment_to_table = 2; + repeated FragmentWorkerSlotMapping worker_slot_mappings = 3; } service ServingService { diff --git a/src/batch/src/executor/join/local_lookup_join.rs b/src/batch/src/executor/join/local_lookup_join.rs index 17b257106fb5b..638fbf5b96b6d 100644 --- a/src/batch/src/executor/join/local_lookup_join.rs +++ b/src/batch/src/executor/join/local_lookup_join.rs @@ -21,7 +21,7 @@ use risingwave_common::buffer::BitmapBuilder; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; use risingwave_common::hash::table_distribution::TableDistribution; use risingwave_common::hash::{ - ExpandedParallelUnitMapping, HashKey, HashKeyDispatcher, ParallelUnitId, VirtualNode, + ExpandedWorkerSlotMapping, HashKey, HashKeyDispatcher, VirtualNode, WorkerSlotId, }; use risingwave_common::memory::MemoryContext; use risingwave_common::types::{DataType, Datum}; @@ -29,7 +29,6 @@ use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::scan_range::ScanRange; use risingwave_common::util::tracing::TracingContext; -use risingwave_common::util::worker_util::get_pu_to_worker_mapping; use risingwave_expr::expr::{build_from_prost, BoxedExpression}; use risingwave_pb::batch_plan::exchange_info::DistributionMode; use risingwave_pb::batch_plan::exchange_source::LocalExecutePlan::Plan; @@ -52,7 +51,7 @@ use crate::task::{BatchTaskContext, ShutdownToken, TaskId}; struct InnerSideExecutorBuilder { table_desc: StorageTableDesc, table_distribution: TableDistribution, - vnode_mapping: ExpandedParallelUnitMapping, + vnode_mapping: ExpandedWorkerSlotMapping, outer_side_key_types: Vec, inner_side_schema: Schema, inner_side_column_ids: Vec, @@ -61,8 +60,8 @@ struct InnerSideExecutorBuilder { context: C, task_id: TaskId, epoch: BatchQueryEpoch, - pu_to_worker_mapping: HashMap, - pu_to_scan_range_mapping: HashMap>, + worker_slot_mapping: HashMap, + worker_slot_to_scan_range_mapping: HashMap>, chunk_size: usize, shutdown_rx: ShutdownToken, next_stage_id: usize, @@ -91,8 +90,8 @@ impl InnerSideExecutorBuilder { /// Creates the `RowSeqScanNode` that will be used for scanning the inner side table /// based on the passed `scan_range` and virtual node. - fn create_row_seq_scan_node(&self, id: &ParallelUnitId) -> Result { - let list = self.pu_to_scan_range_mapping.get(id).unwrap(); + fn create_row_seq_scan_node(&self, id: &WorkerSlotId) -> Result { + let list = self.worker_slot_to_scan_range_mapping.get(id).unwrap(); let mut scan_ranges = vec![]; let mut vnode_bitmap = BitmapBuilder::zeroed(self.vnode_mapping.len()); @@ -114,11 +113,11 @@ impl InnerSideExecutorBuilder { } /// Creates the `PbExchangeSource` using the given `id`. - fn build_prost_exchange_source(&self, id: &ParallelUnitId) -> Result { + fn build_prost_exchange_source(&self, id: &WorkerSlotId) -> Result { let worker = self - .pu_to_worker_mapping + .worker_slot_mapping .get(id) - .context("No worker node found for the given parallel unit id.")?; + .context("No worker node found for the given worker slot id.")?; let local_execute_plan = LocalExecutePlan { plan: Some(PlanFragment { @@ -145,7 +144,7 @@ impl InnerSideExecutorBuilder { // conflict. query_id: self.task_id.query_id.clone(), stage_id: self.task_id.stage_id + 10000 + self.next_stage_id as u32, - task_id: *id, + task_id: (*id).into(), }), output_id: 0, }), @@ -160,7 +159,7 @@ impl InnerSideExecutorBuilder { #[async_trait::async_trait] impl LookupExecutorBuilder for InnerSideExecutorBuilder { fn reset(&mut self) { - self.pu_to_scan_range_mapping = HashMap::new(); + self.worker_slot_to_scan_range_mapping = HashMap::new(); } /// Adds the scan range made from the given `kwy_scalar_impls` into the parallel unit id @@ -191,11 +190,11 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder } let vnode = self.get_virtual_node(&scan_range)?; - let parallel_unit_id = self.vnode_mapping[vnode.to_index()]; + let worker_slot_id = self.vnode_mapping[vnode.to_index()]; let list = self - .pu_to_scan_range_mapping - .entry(parallel_unit_id) + .worker_slot_to_scan_range_mapping + .entry(worker_slot_id) .or_default(); list.push((scan_range, vnode)); @@ -207,7 +206,7 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder async fn build_executor(&mut self) -> Result { self.next_stage_id += 1; let mut sources = vec![]; - for id in self.pu_to_scan_range_mapping.keys() { + for id in self.worker_slot_to_scan_range_mapping.keys() { sources.push(self.build_prost_exchange_source(id)?); } @@ -368,11 +367,26 @@ impl BoxedExecutorBuilder for LocalLookupJoinExecutorBuilder { let null_safe = lookup_join_node.get_null_safe().to_vec(); - let vnode_mapping = lookup_join_node.get_inner_side_vnode_mapping().to_vec(); + let vnode_mapping = lookup_join_node + .get_inner_side_vnode_mapping() + .iter() + .copied() + .map(WorkerSlotId::from) + .collect_vec(); + assert!(!vnode_mapping.is_empty()); let chunk_size = source.context.get_config().developer.chunk_size; + let worker_nodes = lookup_join_node.get_worker_nodes(); + let worker_slot_mapping: HashMap = worker_nodes + .iter() + .flat_map(|worker| { + (0..(worker.parallel_units.len())) + .map(|i| (WorkerSlotId::new(worker.id, i), worker.clone())) + }) + .collect(); + let inner_side_builder = InnerSideExecutorBuilder { table_desc: table_desc.clone(), table_distribution: TableDistribution::new_from_storage_table_desc( @@ -388,11 +402,11 @@ impl BoxedExecutorBuilder for LocalLookupJoinExecutorBuilder { context: source.context().clone(), task_id: source.task_id.clone(), epoch: source.epoch(), - pu_to_worker_mapping: get_pu_to_worker_mapping(lookup_join_node.get_worker_nodes()), - pu_to_scan_range_mapping: HashMap::new(), + worker_slot_to_scan_range_mapping: HashMap::new(), chunk_size, shutdown_rx: source.shutdown_rx.clone(), next_stage_id: 0, + worker_slot_mapping, }; let identity = source.plan_node().get_identity().clone(); diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index 87d94bc1d90b9..b4ad51da3bb41 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -79,7 +79,7 @@ impl StateReporter { #[derive(PartialEq, Eq, Hash, Clone, Debug, Default)] pub struct TaskId { - pub task_id: u32, + pub task_id: u64, pub stage_id: u32, pub query_id: String, } @@ -87,7 +87,7 @@ pub struct TaskId { #[derive(PartialEq, Eq, Hash, Clone, Default)] pub struct TaskOutputId { pub task_id: TaskId, - pub output_id: u32, + pub output_id: u64, } /// More compact formatter compared to derived `fmt::Debug`. diff --git a/src/batch/src/worker_manager/worker_node_manager.rs b/src/batch/src/worker_manager/worker_node_manager.rs index 5b0813186fd1c..8b7dcc42b565a 100644 --- a/src/batch/src/worker_manager/worker_node_manager.rs +++ b/src/batch/src/worker_manager/worker_node_manager.rs @@ -18,8 +18,7 @@ use std::time::Duration; use rand::seq::SliceRandom; use risingwave_common::bail; -use risingwave_common::hash::{ParallelUnitId, ParallelUnitMapping}; -use risingwave_common::util::worker_util::get_pu_to_worker_mapping; +use risingwave_common::hash::{WorkerSlotId, WorkerSlotMapping}; use risingwave_common::vnode_mapping::vnode_placement::place_vnode; use risingwave_pb::common::{WorkerNode, WorkerType}; @@ -36,12 +35,10 @@ pub struct WorkerNodeManager { struct WorkerNodeManagerInner { worker_nodes: Vec, - /// A cache for parallel units to worker nodes. It should be consistent with `worker_nodes`. - pu_to_worker: HashMap, /// fragment vnode mapping info for streaming - streaming_fragment_vnode_mapping: HashMap, + streaming_fragment_vnode_mapping: HashMap, /// fragment vnode mapping info for serving - serving_fragment_vnode_mapping: HashMap, + serving_fragment_vnode_mapping: HashMap, } pub type WorkerNodeManagerRef = Arc; @@ -57,7 +54,6 @@ impl WorkerNodeManager { Self { inner: RwLock::new(WorkerNodeManagerInner { worker_nodes: Default::default(), - pu_to_worker: Default::default(), streaming_fragment_vnode_mapping: Default::default(), serving_fragment_vnode_mapping: Default::default(), }), @@ -68,7 +64,6 @@ impl WorkerNodeManager { /// Used in tests. pub fn mock(worker_nodes: Vec) -> Self { let inner = RwLock::new(WorkerNodeManagerInner { - pu_to_worker: get_pu_to_worker_mapping(&worker_nodes), worker_nodes, streaming_fragment_vnode_mapping: HashMap::new(), serving_fragment_vnode_mapping: HashMap::new(), @@ -120,23 +115,18 @@ impl WorkerNodeManager { *w = node; } } - // Update `pu_to_worker` - write_guard.pu_to_worker = get_pu_to_worker_mapping(&write_guard.worker_nodes); } pub fn remove_worker_node(&self, node: WorkerNode) { let mut write_guard = self.inner.write().unwrap(); write_guard.worker_nodes.retain(|x| x.id != node.id); - - // Update `pu_to_worker` - write_guard.pu_to_worker = get_pu_to_worker_mapping(&write_guard.worker_nodes); } pub fn refresh( &self, nodes: Vec, - streaming_mapping: HashMap, - serving_mapping: HashMap, + streaming_mapping: HashMap, + serving_mapping: HashMap, ) { let mut write_guard = self.inner.write().unwrap(); tracing::debug!("Refresh worker nodes {:?}.", nodes); @@ -149,42 +139,51 @@ impl WorkerNodeManager { serving_mapping.keys() ); write_guard.worker_nodes = nodes; - // Update `pu_to_worker` - write_guard.pu_to_worker = get_pu_to_worker_mapping(&write_guard.worker_nodes); write_guard.streaming_fragment_vnode_mapping = streaming_mapping; write_guard.serving_fragment_vnode_mapping = serving_mapping; } - /// If parallel unit ids is empty, the scheduler may fail to schedule any task and stuck at + /// If worker slot ids is empty, the scheduler may fail to schedule any task and stuck at /// schedule next stage. If we do not return error in this case, needs more complex control /// logic above. Report in this function makes the schedule root fail reason more clear. - pub fn get_workers_by_parallel_unit_ids( + pub fn get_workers_by_worker_slot_ids( &self, - parallel_unit_ids: &[ParallelUnitId], + worker_slot_ids: &[WorkerSlotId], ) -> Result> { - if parallel_unit_ids.is_empty() { + if worker_slot_ids.is_empty() { return Err(BatchError::EmptyWorkerNodes); } let guard = self.inner.read().unwrap(); - let mut workers = Vec::with_capacity(parallel_unit_ids.len()); - for parallel_unit_id in parallel_unit_ids { - match guard.pu_to_worker.get(parallel_unit_id) { - Some(worker) => workers.push(worker.clone()), + let worker_slot_index: HashMap<_, _> = guard + .worker_nodes + .iter() + .flat_map(|worker| { + (0..worker.parallel_units.len()) + .map(move |i| (WorkerSlotId::new(worker.id, i), worker)) + }) + .collect(); + + let mut workers = Vec::with_capacity(worker_slot_ids.len()); + + for worker_slot_id in worker_slot_ids { + match worker_slot_index.get(worker_slot_id) { + Some(worker) => workers.push((*worker).clone()), None => bail!( - "No worker node found for parallel unit id: {}", - parallel_unit_id + "No worker node found for worker slot id: {}", + worker_slot_id ), } } + Ok(workers) } pub fn get_streaming_fragment_mapping( &self, fragment_id: &FragmentId, - ) -> Result { + ) -> Result { self.inner .read() .unwrap() @@ -197,7 +196,7 @@ impl WorkerNodeManager { pub fn insert_streaming_fragment_mapping( &self, fragment_id: FragmentId, - vnode_mapping: ParallelUnitMapping, + vnode_mapping: WorkerSlotMapping, ) { self.inner .write() @@ -210,7 +209,7 @@ impl WorkerNodeManager { pub fn update_streaming_fragment_mapping( &self, fragment_id: FragmentId, - vnode_mapping: ParallelUnitMapping, + vnode_mapping: WorkerSlotMapping, ) { let mut guard = self.inner.write().unwrap(); guard @@ -228,7 +227,7 @@ impl WorkerNodeManager { } /// Returns fragment's vnode mapping for serving. - fn serving_fragment_mapping(&self, fragment_id: FragmentId) -> Result { + fn serving_fragment_mapping(&self, fragment_id: FragmentId) -> Result { self.inner .read() .unwrap() @@ -236,7 +235,7 @@ impl WorkerNodeManager { .ok_or_else(|| BatchError::ServingVnodeMappingNotFound(fragment_id)) } - pub fn set_serving_fragment_mapping(&self, mappings: HashMap) { + pub fn set_serving_fragment_mapping(&self, mappings: HashMap) { let mut guard = self.inner.write().unwrap(); tracing::debug!( "Set serving vnode mapping for fragments {:?}", @@ -247,7 +246,7 @@ impl WorkerNodeManager { pub fn upsert_serving_fragment_mapping( &self, - mappings: HashMap, + mappings: HashMap, ) { let mut guard = self.inner.write().unwrap(); tracing::debug!( @@ -299,7 +298,7 @@ impl WorkerNodeManager { } impl WorkerNodeManagerInner { - fn get_serving_fragment_mapping(&self, fragment_id: FragmentId) -> Option { + fn get_serving_fragment_mapping(&self, fragment_id: FragmentId) -> Option { self.serving_fragment_vnode_mapping .get(&fragment_id) .cloned() @@ -342,7 +341,7 @@ impl WorkerNodeSelector { .sum() } - pub fn fragment_mapping(&self, fragment_id: FragmentId) -> Result { + pub fn fragment_mapping(&self, fragment_id: FragmentId) -> Result { if self.enable_barrier_read { self.manager.get_streaming_fragment_mapping(&fragment_id) } else { diff --git a/src/common/common_service/src/observer_manager.rs b/src/common/common_service/src/observer_manager.rs index 3f18be7697520..a611c40aebc68 100644 --- a/src/common/common_service/src/observer_manager.rs +++ b/src/common/common_service/src/observer_manager.rs @@ -145,9 +145,6 @@ where | Info::Function(_) => { notification.version > info.version.as_ref().unwrap().catalog_version } - Info::ParallelUnitMapping(_) => { - notification.version > info.version.as_ref().unwrap().parallel_unit_mapping_version - } Info::Node(_) => { notification.version > info.version.as_ref().unwrap().worker_node_version } @@ -157,10 +154,18 @@ where Info::HummockSnapshot(_) => true, Info::MetaBackupManifestId(_) => true, Info::SystemParams(_) | Info::SessionParam(_) => true, - Info::ServingParallelUnitMappings(_) => true, Info::Snapshot(_) | Info::HummockWriteLimits(_) => unreachable!(), Info::HummockStats(_) => true, Info::Recovery(_) => true, + Info::StreamingWorkerSlotMapping(_) => { + notification.version + > info + .version + .as_ref() + .unwrap() + .streaming_worker_slot_mapping_version + } + Info::ServingWorkerSlotMappings(_) => true, }); self.observer_states diff --git a/src/common/src/hash/consistent_hash/mapping.rs b/src/common/src/hash/consistent_hash/mapping.rs index 07e62b7eac275..c057cf847c7db 100644 --- a/src/common/src/hash/consistent_hash/mapping.rs +++ b/src/common/src/hash/consistent_hash/mapping.rs @@ -12,14 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; -use std::fmt::Debug; +use std::collections::{BTreeSet, HashMap}; +use std::fmt::{Debug, Display, Formatter}; use std::hash::Hash; use std::ops::Index; use educe::Educe; use itertools::Itertools; -use risingwave_pb::common::{ParallelUnit, ParallelUnitMapping as ParallelUnitMappingProto}; +use risingwave_pb::common::{ + ParallelUnit, ParallelUnitMapping as ParallelUnitMappingProto, PbWorkerSlotMapping, +}; use risingwave_pb::stream_plan::ActorMapping as ActorMappingProto; use super::bitmap::VnodeBitmapExt; @@ -31,6 +33,41 @@ use crate::util::iter_util::ZipEqDebug; // TODO: find a better place for this. pub type ActorId = u32; +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Ord, PartialOrd)] +pub struct WorkerSlotId(u64); + +impl WorkerSlotId { + pub fn worker_id(&self) -> u32 { + (self.0 >> 32) as u32 + } + + pub fn slot_idx(&self) -> u32 { + self.0 as u32 + } + + pub fn new(worker_id: u32, slot_idx: usize) -> Self { + Self((worker_id as u64) << 32 | slot_idx as u64) + } +} + +impl From for u64 { + fn from(id: WorkerSlotId) -> Self { + id.0 + } +} + +impl From for WorkerSlotId { + fn from(id: u64) -> Self { + Self(id) + } +} + +impl Display for WorkerSlotId { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.write_fmt(format_args!("[{}:{}]", self.worker_id(), self.slot_idx())) + } +} + /// Trait for items that can be used as keys in [`VnodeMapping`]. pub trait VnodeMappingItem { /// The type of the item. @@ -254,6 +291,12 @@ pub mod marker { impl VnodeMappingItem for ParallelUnit { type Item = ParallelUnitId; } + + /// A marker type for items of [`WorkerSlotId`]. + pub struct WorkerSlot; + impl VnodeMappingItem for WorkerSlot { + type Item = WorkerSlotId; + } } /// A mapping from [`VirtualNode`] to [`ActorId`]. @@ -266,6 +309,11 @@ pub type ParallelUnitMapping = VnodeMapping; /// An expanded mapping from [`VirtualNode`] to [`ParallelUnitId`]. pub type ExpandedParallelUnitMapping = ExpandedMapping; +/// A mapping from [`VirtualNode`] to [`WorkerSlotId`]. +pub type WorkerSlotMapping = VnodeMapping; +/// An expanded mapping from [`VirtualNode`] to [`WorkerSlotId`]. +pub type ExpandedWorkerSlotMapping = ExpandedMapping; + impl ActorMapping { /// Transform this actor mapping to a parallel unit mapping, essentially `transform`. pub fn to_parallel_unit(&self, to_map: &M) -> ParallelUnitMapping @@ -293,6 +341,30 @@ impl ActorMapping { } } +impl WorkerSlotMapping { + /// Create a uniform worker mapping from the given worker ids + pub fn build_from_ids(worker_slot_ids: &[WorkerSlotId]) -> Self { + Self::new_uniform(worker_slot_ids.iter().cloned()) + } + + /// Create a worker mapping from the protobuf representation. + pub fn from_protobuf(proto: &PbWorkerSlotMapping) -> Self { + assert_eq!(proto.original_indices.len(), proto.data.len()); + Self { + original_indices: proto.original_indices.clone(), + data: proto.data.iter().map(|&id| WorkerSlotId(id)).collect(), + } + } + + /// Convert this worker mapping to the protobuf representation. + pub fn to_protobuf(&self) -> PbWorkerSlotMapping { + PbWorkerSlotMapping { + original_indices: self.original_indices.clone(), + data: self.data.iter().map(|id| id.0).collect(), + } + } +} + impl ParallelUnitMapping { /// Create a uniform parallel unit mapping from the given parallel units, essentially /// `new_uniform`. @@ -310,6 +382,28 @@ impl ParallelUnitMapping { self.transform(to_map) } + /// Transform this parallel unit mapping to an worker mapping, essentially `transform`. + pub fn to_worker_slot(&self, to_map: &HashMap) -> WorkerSlotMapping { + let mut worker_to_parallel_units = HashMap::<_, BTreeSet<_>>::new(); + for (parallel_unit_id, worker_id) in to_map { + worker_to_parallel_units + .entry(*worker_id) + .or_default() + .insert(*parallel_unit_id); + } + + let mut parallel_unit_to_worker_slot = HashMap::with_capacity(to_map.len()); + + for (worker_id, parallel_unit_ids) in worker_to_parallel_units { + for (index, ¶llel_unit_id) in parallel_unit_ids.iter().enumerate() { + parallel_unit_to_worker_slot + .insert(parallel_unit_id, WorkerSlotId::new(worker_id, index)); + } + } + + self.transform(¶llel_unit_to_worker_slot) + } + /// Create a parallel unit mapping from the protobuf representation. pub fn from_protobuf(proto: &ParallelUnitMappingProto) -> Self { assert_eq!(proto.original_indices.len(), proto.data.len()); diff --git a/src/common/src/util/worker_util.rs b/src/common/src/util/worker_util.rs index 893cd95ecbbbb..80ecd3b822536 100644 --- a/src/common/src/util/worker_util.rs +++ b/src/common/src/util/worker_util.rs @@ -12,23 +12,4 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; - -use risingwave_pb::common::WorkerNode; - -use crate::hash::ParallelUnitId; - pub type WorkerNodeId = u32; - -pub fn get_pu_to_worker_mapping(nodes: &[WorkerNode]) -> HashMap { - let mut pu_to_worker = HashMap::new(); - - for node in nodes { - for pu in &node.parallel_units { - let res = pu_to_worker.insert(pu.id, node.clone()); - assert!(res.is_none(), "duplicate parallel unit id"); - } - } - - pu_to_worker -} diff --git a/src/common/src/vnode_mapping/vnode_placement.rs b/src/common/src/vnode_mapping/vnode_placement.rs index 49f45d66512eb..036cfebe792bb 100644 --- a/src/common/src/vnode_mapping/vnode_placement.rs +++ b/src/common/src/vnode_mapping/vnode_placement.rs @@ -13,46 +13,48 @@ // limitations under the License. use std::collections::{HashMap, HashSet, LinkedList, VecDeque}; +use std::ops::BitOrAssign; use itertools::Itertools; use num_integer::Integer; +use risingwave_common::hash::WorkerSlotId; use risingwave_pb::common::WorkerNode; use crate::buffer::{Bitmap, BitmapBuilder}; -use crate::hash::{ParallelUnitId, ParallelUnitMapping, VirtualNode}; +use crate::hash::{VirtualNode, WorkerSlotMapping}; /// Calculate a new vnode mapping, keeping locality and balance on a best effort basis. /// The strategy is similar to `rebalance_actor_vnode` used in meta node, but is modified to /// consider `max_parallelism` too. pub fn place_vnode( - hint_pu_mapping: Option<&ParallelUnitMapping>, - new_workers: &[WorkerNode], + hint_worker_slot_mapping: Option<&WorkerSlotMapping>, + workers: &[WorkerNode], max_parallelism: Option, -) -> Option { - // Get all serving parallel units from all available workers, grouped by worker id and ordered - // by parallel unit id in each group. - let mut new_pus: LinkedList<_> = new_workers +) -> Option { + // Get all serving worker slots from all available workers, grouped by worker id and ordered + // by worker slot id in each group. + let mut worker_slots: LinkedList<_> = workers .iter() .filter(|w| w.property.as_ref().map_or(false, |p| p.is_serving)) .sorted_by_key(|w| w.id) - .map(|w| w.parallel_units.clone().into_iter().sorted_by_key(|p| p.id)) + .map(|w| (0..w.parallel_units.len()).map(|idx| WorkerSlotId::new(w.id, idx))) .collect(); - // Set serving parallelism to the minimum of total number of parallel units, specified + // Set serving parallelism to the minimum of total number of worker slots, specified // `max_parallelism` and total number of virtual nodes. let serving_parallelism = std::cmp::min( - new_pus.iter().map(|pus| pus.len()).sum(), + worker_slots.iter().map(|slots| slots.len()).sum(), std::cmp::min(max_parallelism.unwrap_or(usize::MAX), VirtualNode::COUNT), ); - // Select `serving_parallelism` parallel units in a round-robin fashion, to distribute workload + // Select `serving_parallelism` worker slots in a round-robin fashion, to distribute workload // evenly among workers. - let mut selected_pu_ids = Vec::new(); - while !new_pus.is_empty() { - new_pus - .extract_if(|ps| { - if let Some(p) = ps.next() { - selected_pu_ids.push(p.id); + let mut selected_slots = Vec::new(); + while !worker_slots.is_empty() { + worker_slots + .extract_if(|slots| { + if let Some(slot) = slots.next() { + selected_slots.push(slot); false } else { true @@ -60,57 +62,61 @@ pub fn place_vnode( }) .for_each(drop); } - selected_pu_ids.drain(serving_parallelism..); - let selected_pu_id_set: HashSet = selected_pu_ids.iter().cloned().collect(); - if selected_pu_id_set.is_empty() { + selected_slots.drain(serving_parallelism..); + let selected_slots_set: HashSet = selected_slots.iter().cloned().collect(); + if selected_slots_set.is_empty() { return None; } - // Calculate balance for each selected parallel unit. Initially, each parallel unit is assigned + // Calculate balance for each selected worker slot. Initially, each worker slot is assigned // no vnodes. Thus its negative balance means that many vnodes should be assigned to it later. - // `is_temp` is a mark for a special temporary parallel unit, only to simplify implementation. + // `is_temp` is a mark for a special temporary worker slot, only to simplify implementation. #[derive(Debug)] struct Balance { - pu_id: ParallelUnitId, + slot: WorkerSlotId, balance: i32, builder: BitmapBuilder, is_temp: bool, } - let (expected, mut remain) = VirtualNode::COUNT.div_rem(&selected_pu_ids.len()); - let mut balances: HashMap = HashMap::default(); - for pu_id in &selected_pu_ids { + + let (expected, mut remain) = VirtualNode::COUNT.div_rem(&selected_slots.len()); + let mut balances: HashMap = HashMap::default(); + + for slot in &selected_slots { let mut balance = Balance { - pu_id: *pu_id, + slot: *slot, balance: -(expected as i32), builder: BitmapBuilder::zeroed(VirtualNode::COUNT), is_temp: false, }; + if remain > 0 { balance.balance -= 1; remain -= 1; } - balances.insert(*pu_id, balance); + balances.insert(*slot, balance); } - // Now to maintain affinity, if a hint has been provided via `hint_pu_mapping`, follow + // Now to maintain affinity, if a hint has been provided via `hint_worker_slot_mapping`, follow // that mapping to adjust balances. - let mut temp_pu = Balance { - pu_id: 0, // This id doesn't matter for `temp_pu`. It's distinguishable via `is_temp`. + let mut temp_slot = Balance { + slot: WorkerSlotId::new(0u32, usize::MAX), /* This id doesn't matter for `temp_slot`. It's distinguishable via `is_temp`. */ balance: 0, builder: BitmapBuilder::zeroed(VirtualNode::COUNT), is_temp: true, }; - match hint_pu_mapping { - Some(hint_pu_mapping) => { - for (vnode, pu_id) in hint_pu_mapping.iter_with_vnode() { - let b = if selected_pu_id_set.contains(&pu_id) { - // Assign vnode to the same parallel unit as hint. - balances.get_mut(&pu_id).unwrap() + match hint_worker_slot_mapping { + Some(hint_worker_slot_mapping) => { + for (vnode, worker_slot) in hint_worker_slot_mapping.iter_with_vnode() { + let b = if selected_slots_set.contains(&worker_slot) { + // Assign vnode to the same worker slot as hint. + balances.get_mut(&worker_slot).unwrap() } else { - // Assign vnode that doesn't belong to any parallel unit to `temp_pu` + // Assign vnode that doesn't belong to any worker slot to `temp_slot` // temporarily. They will be reassigned later. - &mut temp_pu + &mut temp_slot }; + b.balance += 1; b.builder.set(vnode.to_index(), true); } @@ -118,31 +124,33 @@ pub fn place_vnode( None => { // No hint is provided, assign all vnodes to `temp_pu`. for vnode in VirtualNode::all() { - temp_pu.balance += 1; - temp_pu.builder.set(vnode.to_index(), true); + temp_slot.balance += 1; + temp_slot.builder.set(vnode.to_index(), true); } } } - // The final step is to move vnodes from parallel units with positive balance to parallel units - // with negative balance, until all parallel units are of 0 balance. - // A double-ended queue with parallel units ordered by balance in descending order is consumed: - // 1. Peek 2 parallel units from front and back. + // The final step is to move vnodes from worker slots with positive balance to worker slots + // with negative balance, until all worker slots are of 0 balance. + // A double-ended queue with worker slots ordered by balance in descending order is consumed: + // 1. Peek 2 worker slots from front and back. // 2. It any of them is of 0 balance, pop it and go to step 1. // 3. Otherwise, move vnodes from front to back. let mut balances: VecDeque<_> = balances .into_values() - .chain(std::iter::once(temp_pu)) + .chain(std::iter::once(temp_slot)) .sorted_by_key(|b| b.balance) .rev() .collect(); - let mut results: HashMap = HashMap::default(); + + let mut results: HashMap = HashMap::default(); + while !balances.is_empty() { if balances.len() == 1 { let single = balances.pop_front().unwrap(); assert_eq!(single.balance, 0); if !single.is_temp { - results.insert(single.pu_id, single.builder.finish()); + results.insert(single.slot, single.builder.finish()); } break; } @@ -166,32 +174,42 @@ pub fn place_vnode( if src.balance != 0 { balances.push_front(src); } else if !src.is_temp { - results.insert(src.pu_id, src.builder.finish()); + results.insert(src.slot, src.builder.finish()); } if dst.balance != 0 { balances.push_back(dst); } else if !dst.is_temp { - results.insert(dst.pu_id, dst.builder.finish()); + results.insert(dst.slot, dst.builder.finish()); } } - Some(ParallelUnitMapping::from_bitmaps(&results)) + let mut worker_result = HashMap::new(); + + for (worker_slot, bitmap) in results { + worker_result + .entry(worker_slot) + .or_insert(BitmapBuilder::zeroed(VirtualNode::COUNT).finish()) + .bitor_assign(&bitmap); + } + + Some(WorkerSlotMapping::from_bitmaps(&worker_result)) } #[cfg(test)] mod tests { use std::collections::HashMap; + use risingwave_common::hash::WorkerSlotMapping; use risingwave_pb::common::worker_node::Property; use risingwave_pb::common::{ParallelUnit, WorkerNode}; - use crate::hash::{ParallelUnitId, ParallelUnitMapping, VirtualNode}; + use crate::hash::{ParallelUnitId, VirtualNode}; use crate::vnode_mapping::vnode_placement::place_vnode; - #[test] fn test_place_vnode() { assert_eq!(VirtualNode::COUNT, 256); + let mut pu_id_counter: ParallelUnitId = 0; let mut pu_to_worker: HashMap = Default::default(); let serving_property = Property { @@ -216,13 +234,13 @@ mod tests { results }; - let count_same_vnode_mapping = |pm1: &ParallelUnitMapping, pm2: &ParallelUnitMapping| { - assert_eq!(pm1.len(), 256); - assert_eq!(pm2.len(), 256); + let count_same_vnode_mapping = |wm1: &WorkerSlotMapping, wm2: &WorkerSlotMapping| { + assert_eq!(wm1.len(), 256); + assert_eq!(wm2.len(), 256); let mut count: usize = 0; for idx in 0..VirtualNode::COUNT { let vnode = VirtualNode::from_index(idx); - if pm1.get(vnode) == pm2.get(vnode) { + if wm1.get(vnode) == wm2.get(vnode) { count += 1; } } @@ -235,29 +253,32 @@ mod tests { property: Some(serving_property.clone()), ..Default::default() }; + assert!( place_vnode(None, &[worker_1.clone()], Some(0)).is_none(), "max_parallelism should >= 0" ); - let re_pu_mapping_2 = place_vnode(None, &[worker_1.clone()], None).unwrap(); - assert_eq!(re_pu_mapping_2.iter_unique().count(), 1); + let re_worker_mapping_2 = place_vnode(None, &[worker_1.clone()], None).unwrap(); + assert_eq!(re_worker_mapping_2.iter_unique().count(), 1); + let worker_2 = WorkerNode { id: 2, parallel_units: gen_pus_for_worker(2, 50, &mut pu_to_worker), property: Some(serving_property.clone()), ..Default::default() }; - let re_pu_mapping = place_vnode( - Some(&re_pu_mapping_2), + + let re_worker_mapping = place_vnode( + Some(&re_worker_mapping_2), &[worker_1.clone(), worker_2.clone()], None, ) .unwrap(); - assert_eq!(re_pu_mapping.iter_unique().count(), 51); + assert_eq!(re_worker_mapping.iter_unique().count(), 51); // 1 * 256 + 0 -> 51 * 5 + 1 - let score = count_same_vnode_mapping(&re_pu_mapping_2, &re_pu_mapping); + let score = count_same_vnode_mapping(&re_worker_mapping_2, &re_worker_mapping); assert!(score >= 5); let worker_3 = WorkerNode { @@ -267,7 +288,7 @@ mod tests { ..Default::default() }; let re_pu_mapping_2 = place_vnode( - Some(&re_pu_mapping), + Some(&re_worker_mapping), &[worker_1.clone(), worker_2.clone(), worker_3.clone()], None, ) @@ -276,7 +297,7 @@ mod tests { // limited by total pu number assert_eq!(re_pu_mapping_2.iter_unique().count(), 111); // 51 * 5 + 1 -> 111 * 2 + 34 - let score = count_same_vnode_mapping(&re_pu_mapping_2, &re_pu_mapping); + let score = count_same_vnode_mapping(&re_pu_mapping_2, &re_worker_mapping); assert!(score >= (2 + 50 * 2)); let re_pu_mapping = place_vnode( Some(&re_pu_mapping_2), diff --git a/src/ctl/src/cmd_impl/meta/serving.rs b/src/ctl/src/cmd_impl/meta/serving.rs index c6c5d3cf81985..cd8b1be50e206 100644 --- a/src/ctl/src/cmd_impl/meta/serving.rs +++ b/src/ctl/src/cmd_impl/meta/serving.rs @@ -16,30 +16,26 @@ use std::collections::HashMap; use comfy_table::{Row, Table}; use itertools::Itertools; -use risingwave_common::hash::{ParallelUnitId, VirtualNode}; -use risingwave_pb::common::{WorkerNode, WorkerType}; +use risingwave_common::hash::VirtualNode; +use risingwave_pb::common::WorkerType; use crate::CtlContext; pub async fn list_serving_fragment_mappings(context: &CtlContext) -> anyhow::Result<()> { let meta_client = context.meta_client().await?; let mappings = meta_client.list_serving_vnode_mappings().await?; - let workers = meta_client + let workers: HashMap<_, _> = meta_client .list_worker_nodes(Some(WorkerType::ComputeNode)) - .await?; - let mut pu_to_worker: HashMap = HashMap::new(); - for w in &workers { - for pu in &w.parallel_units { - pu_to_worker.insert(pu.id, w); - } - } + .await? + .into_iter() + .map(|worker| (worker.id, worker)) + .collect(); let mut table = Table::new(); table.set_header({ let mut row = Row::new(); row.add_cell("Table Id".into()); row.add_cell("Fragment Id".into()); - row.add_cell("Parallel Unit Id".into()); row.add_cell("Virtual Node".into()); row.add_cell("Worker".into()); row @@ -48,28 +44,25 @@ pub async fn list_serving_fragment_mappings(context: &CtlContext) -> anyhow::Res let rows = mappings .iter() .flat_map(|(fragment_id, (table_id, mapping))| { - let mut pu_vnodes: HashMap> = HashMap::new(); - for (vnode, pu) in mapping.iter_with_vnode() { - pu_vnodes.entry(pu).or_default().push(vnode); + let mut worker_nodes: HashMap> = HashMap::new(); + for (vnode, worker_slot_id) in mapping.iter_with_vnode() { + worker_nodes + .entry(worker_slot_id.worker_id()) + .or_default() + .push(vnode); } - pu_vnodes.into_iter().map(|(pu_id, vnodes)| { - ( - *table_id, - *fragment_id, - pu_id, - vnodes, - pu_to_worker.get(&pu_id), - ) + worker_nodes.into_iter().map(|(worker_id, vnodes)| { + (*table_id, *fragment_id, vnodes, workers.get(&worker_id)) }) }) .collect_vec(); - for (table_id, fragment_id, pu_id, vnodes, worker) in - rows.into_iter().sorted_by_key(|(t, f, p, ..)| (*t, *f, *p)) + + for (table_id, fragment_id, vnodes, worker) in + rows.into_iter().sorted_by_key(|(t, f, ..)| (*t, *f)) { let mut row = Row::new(); row.add_cell(table_id.into()); row.add_cell(fragment_id.into()); - row.add_cell(pu_id.into()); row.add_cell( format!( "{} in total: {}", diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index ddf6ca489bf0c..f864f9608bdba 100644 --- a/src/frontend/src/observer/observer_manager.rs +++ b/src/frontend/src/observer/observer_manager.rs @@ -19,7 +19,7 @@ use itertools::Itertools; use parking_lot::RwLock; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeManagerRef; use risingwave_common::catalog::CatalogVersion; -use risingwave_common::hash::ParallelUnitMapping; +use risingwave_common::hash::WorkerSlotMapping; use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; use risingwave_common_service::observer_manager::{ObserverState, SubscribeFrontend}; @@ -27,7 +27,7 @@ use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::meta::relation::RelationInfo; use risingwave_pb::meta::subscribe_response::{Info, Operation}; -use risingwave_pb::meta::{FragmentParallelUnitMapping, MetaSnapshot, SubscribeResponse}; +use risingwave_pb::meta::{FragmentWorkerSlotMapping, MetaSnapshot, SubscribeResponse}; use risingwave_rpc_client::ComputeClientPoolRef; use tokio::sync::watch::Sender; @@ -72,7 +72,6 @@ impl ObserverState for FrontendObserverNode { Info::User(_) => { self.handle_user_notification(resp); } - Info::ParallelUnitMapping(_) => self.handle_fragment_mapping_notification(resp), Info::Snapshot(_) => { panic!( "receiving a snapshot in the middle is unsupported now {:?}", @@ -103,8 +102,9 @@ impl ObserverState for FrontendObserverNode { Info::HummockStats(stats) => { self.handle_table_stats_notification(stats); } - Info::ServingParallelUnitMappings(m) => { - self.handle_fragment_serving_mapping_notification(m.mappings, resp.operation()); + Info::StreamingWorkerSlotMapping(_) => self.handle_fragment_mapping_notification(resp), + Info::ServingWorkerSlotMappings(m) => { + self.handle_fragment_serving_mapping_notification(m.mappings, resp.operation()) } Info::Recovery(_) => { self.compute_client_pool.invalidate_all(); @@ -133,13 +133,13 @@ impl ObserverState for FrontendObserverNode { functions, connections, users, - parallel_unit_mappings, - serving_parallel_unit_mappings, nodes, hummock_snapshot, hummock_version: _, meta_backup_manifest_id: _, hummock_write_limits: _, + streaming_worker_slot_mappings, + serving_worker_slot_mappings, session_params, version, } = snapshot; @@ -177,10 +177,11 @@ impl ObserverState for FrontendObserverNode { for user in users { user_guard.create_user(user) } + self.worker_node_manager.refresh( nodes, - convert_pu_mapping(¶llel_unit_mappings), - convert_pu_mapping(&serving_parallel_unit_mappings), + convert_worker_slot_mapping(&streaming_worker_slot_mappings), + convert_worker_slot_mapping(&serving_worker_slot_mappings), ); self.hummock_snapshot_manager .update(hummock_snapshot.unwrap()); @@ -387,11 +388,11 @@ impl FrontendObserverNode { return; }; match info { - Info::ParallelUnitMapping(parallel_unit_mapping) => { - let fragment_id = parallel_unit_mapping.fragment_id; + Info::StreamingWorkerSlotMapping(streaming_worker_slot_mapping) => { + let fragment_id = streaming_worker_slot_mapping.fragment_id; let mapping = || { - ParallelUnitMapping::from_protobuf( - parallel_unit_mapping.mapping.as_ref().unwrap(), + WorkerSlotMapping::from_protobuf( + streaming_worker_slot_mapping.mapping.as_ref().unwrap(), ) }; @@ -417,20 +418,20 @@ impl FrontendObserverNode { fn handle_fragment_serving_mapping_notification( &mut self, - mappings: Vec, + mappings: Vec, op: Operation, ) { match op { Operation::Add | Operation::Update => { self.worker_node_manager - .upsert_serving_fragment_mapping(convert_pu_mapping(&mappings)); + .upsert_serving_fragment_mapping(convert_worker_slot_mapping(&mappings)); } Operation::Delete => self.worker_node_manager.remove_serving_fragment_mapping( &mappings.into_iter().map(|m| m.fragment_id).collect_vec(), ), Operation::Snapshot => { self.worker_node_manager - .set_serving_fragment_mapping(convert_pu_mapping(&mappings)); + .set_serving_fragment_mapping(convert_worker_slot_mapping(&mappings)); } _ => panic!("receive an unsupported notify {:?}", op), } @@ -470,17 +471,17 @@ impl FrontendObserverNode { } } -fn convert_pu_mapping( - parallel_unit_mappings: &[FragmentParallelUnitMapping], -) -> HashMap { - parallel_unit_mappings +fn convert_worker_slot_mapping( + worker_slot_mappings: &[FragmentWorkerSlotMapping], +) -> HashMap { + worker_slot_mappings .iter() .map( - |FragmentParallelUnitMapping { + |FragmentWorkerSlotMapping { fragment_id, mapping, }| { - let mapping = ParallelUnitMapping::from_protobuf(mapping.as_ref().unwrap()); + let mapping = WorkerSlotMapping::from_protobuf(mapping.as_ref().unwrap()); (*fragment_id, mapping) }, ) diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs index 358ad934a9ccf..cf02daac47d83 100644 --- a/src/frontend/src/optimizer/property/distribution.rs +++ b/src/frontend/src/optimizer/property/distribution.rs @@ -51,7 +51,7 @@ use generic::PhysicalPlanRef; use itertools::Itertools; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; use risingwave_common::catalog::{FieldDisplay, Schema, TableId}; -use risingwave_common::hash::ParallelUnitId; +use risingwave_common::hash::WorkerSlotId; use risingwave_pb::batch_plan::exchange_info::{ ConsistentHashInfo, Distribution as DistributionPb, DistributionMode, HashInfo, }; @@ -148,14 +148,17 @@ impl Distribution { let vnode_mapping = worker_node_manager .fragment_mapping(Self::get_fragment_id(catalog_reader, table_id)?)?; - let pu2id_map: HashMap = vnode_mapping + let worker_slot_to_id_map: HashMap = vnode_mapping .iter_unique() .enumerate() - .map(|(i, pu)| (pu, i as u32)) + .map(|(i, worker_slot_id)| (worker_slot_id, i as u32)) .collect(); Some(DistributionPb::ConsistentHashInfo(ConsistentHashInfo { - vmap: vnode_mapping.iter().map(|x| pu2id_map[&x]).collect_vec(), + vmap: vnode_mapping + .iter() + .map(|id| worker_slot_to_id_map[&id]) + .collect_vec(), key: key.iter().map(|num| *num as u32).collect(), })) } diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index b556bc3af6c85..165bdcee6476b 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -476,7 +476,7 @@ pub(crate) mod tests { ColumnCatalog, ColumnDesc, ConflictBehavior, CreateType, StreamJobStatus, DEFAULT_SUPER_USER_ID, }; - use risingwave_common::hash::ParallelUnitMapping; + use risingwave_common::hash::{WorkerSlotId, WorkerSlotMapping}; use risingwave_common::types::DataType; use risingwave_pb::common::worker_node::Property; use risingwave_pb::common::{HostAddress, ParallelUnit, WorkerNode, WorkerType}; @@ -721,10 +721,12 @@ pub(crate) mod tests { let workers = vec![worker1, worker2, worker3]; let worker_node_manager = Arc::new(WorkerNodeManager::mock(workers)); let worker_node_selector = WorkerNodeSelector::new(worker_node_manager.clone(), false); - worker_node_manager - .insert_streaming_fragment_mapping(0, ParallelUnitMapping::new_single(0)); + worker_node_manager.insert_streaming_fragment_mapping( + 0, + WorkerSlotMapping::new_single(WorkerSlotId::new(0, 0)), + ); worker_node_manager.set_serving_fragment_mapping( - vec![(0, ParallelUnitMapping::new_single(0))] + vec![(0, WorkerSlotMapping::new_single(WorkerSlotId::new(0, 0)))] .into_iter() .collect(), ); diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index 59294169220e7..c727d2e61b1ff 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -32,7 +32,7 @@ use risingwave_batch::executor::ExecutorBuilder; use risingwave_batch::task::{ShutdownMsg, ShutdownSender, ShutdownToken, TaskId as TaskIdBatch}; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; use risingwave_common::array::DataChunk; -use risingwave_common::hash::ParallelUnitMapping; +use risingwave_common::hash::WorkerSlotMapping; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_connector::source::SplitMetaData; @@ -172,7 +172,7 @@ impl StageExecution { ctx: ExecutionContextRef, ) -> Self { let tasks = (0..stage.parallelism.unwrap()) - .map(|task_id| (task_id, TaskStatusHolder::new(task_id))) + .map(|task_id| (task_id as u64, TaskStatusHolder::new(task_id as u64))) .collect(); Self { @@ -289,7 +289,7 @@ impl StageExecution { /// /// When this method is called, all tasks should have been scheduled, and their `worker_node` /// should have been set. - pub fn all_exchange_sources_for(&self, output_id: u32) -> Vec { + pub fn all_exchange_sources_for(&self, output_id: u64) -> Vec { self.tasks .iter() .map(|(task_id, status_holder)| { @@ -353,12 +353,13 @@ impl StageRunner { // We let each task read one partition by setting the `vnode_ranges` of the scan node in // the task. // We schedule the task to the worker node that owns the data partition. - let parallel_unit_ids = vnode_bitmaps.keys().cloned().collect_vec(); + let worker_slot_ids = vnode_bitmaps.keys().cloned().collect_vec(); let workers = self .worker_node_manager .manager - .get_workers_by_parallel_unit_ids(¶llel_unit_ids)?; - for (i, (parallel_unit_id, worker)) in parallel_unit_ids + .get_workers_by_worker_slot_ids(&worker_slot_ids)?; + + for (i, (worker_slot_id, worker)) in worker_slot_ids .into_iter() .zip_eq_fast(workers.into_iter()) .enumerate() @@ -366,11 +367,11 @@ impl StageRunner { let task_id = TaskIdPb { query_id: self.stage.query_id.id.clone(), stage_id: self.stage.id, - task_id: i as u32, + task_id: i as u64, }; - let vnode_ranges = vnode_bitmaps[¶llel_unit_id].clone(); + let vnode_ranges = vnode_bitmaps[&worker_slot_id].clone(); let plan_fragment = - self.create_plan_fragment(i as u32, Some(PartitionInfo::Table(vnode_ranges))); + self.create_plan_fragment(i as u64, Some(PartitionInfo::Table(vnode_ranges))); futures.push(self.schedule_task( task_id, plan_fragment, @@ -391,10 +392,10 @@ impl StageRunner { let task_id = TaskIdPb { query_id: self.stage.query_id.id.clone(), stage_id: self.stage.id, - task_id: id as u32, + task_id: id as u64, }; let plan_fragment = self - .create_plan_fragment(id as u32, Some(PartitionInfo::Source(split.to_vec()))); + .create_plan_fragment(id as u64, Some(PartitionInfo::Source(split.to_vec()))); let worker = self.choose_worker(&plan_fragment, id as u32, self.stage.dml_table_id)?; futures.push(self.schedule_task( @@ -409,9 +410,9 @@ impl StageRunner { let task_id = TaskIdPb { query_id: self.stage.query_id.id.clone(), stage_id: self.stage.id, - task_id: id, + task_id: id as u64, }; - let plan_fragment = self.create_plan_fragment(id, None); + let plan_fragment = self.create_plan_fragment(id as u64, None); let worker = self.choose_worker(&plan_fragment, id, self.stage.dml_table_id)?; futures.push(self.schedule_task( task_id, @@ -682,7 +683,7 @@ impl StageRunner { fn get_table_dml_vnode_mapping( &self, table_id: &TableId, - ) -> SchedulerResult { + ) -> SchedulerResult { let guard = self.catalog_reader.read_guard(); let table = guard @@ -711,11 +712,11 @@ impl StageRunner { if let Some(table_id) = dml_table_id { let vnode_mapping = self.get_table_dml_vnode_mapping(&table_id)?; - let parallel_unit_ids = vnode_mapping.iter_unique().collect_vec(); + let worker_ids = vnode_mapping.iter_unique().collect_vec(); let candidates = self .worker_node_manager .manager - .get_workers_by_parallel_unit_ids(¶llel_unit_ids)?; + .get_workers_by_worker_slot_ids(&worker_ids)?; if candidates.is_empty() { return Err(BatchError::EmptyWorkerNodes.into()); } @@ -741,17 +742,17 @@ impl StageRunner { .table_id .into(), )?; - let id2pu_vec = self + let id_to_worker_slots = self .worker_node_manager .fragment_mapping(fragment_id)? .iter_unique() .collect_vec(); - let pu = id2pu_vec[task_id as usize]; + let worker_slot_id = id_to_worker_slots[task_id as usize]; let candidates = self .worker_node_manager .manager - .get_workers_by_parallel_unit_ids(&[pu])?; + .get_workers_by_worker_slot_ids(&[worker_slot_id])?; if candidates.is_empty() { return Err(BatchError::EmptyWorkerNodes.into()); } diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index 0ff7540a79d72..89104cc895f77 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -30,7 +30,7 @@ use risingwave_batch::task::{ShutdownToken, TaskId}; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; use risingwave_common::array::DataChunk; use risingwave_common::bail; -use risingwave_common::hash::ParallelUnitMapping; +use risingwave_common::hash::WorkerSlotMapping; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::tracing::{InstrumentStream, TracingContext}; use risingwave_connector::source::SplitMetaData; @@ -313,12 +313,12 @@ impl LocalQueryExecution { // Similar to the distributed case (StageRunner::schedule_tasks). // Set `vnode_ranges` of the scan node in `local_execute_plan` of each // `exchange_source`. - let (parallel_unit_ids, vnode_bitmaps): (Vec<_>, Vec<_>) = + let (worker_ids, vnode_bitmaps): (Vec<_>, Vec<_>) = vnode_bitmaps.clone().into_iter().unzip(); let workers = self .worker_node_manager .manager - .get_workers_by_parallel_unit_ids(¶llel_unit_ids)?; + .get_workers_by_worker_slot_ids(&worker_ids)?; for (idx, (worker_node, partition)) in (workers.into_iter().zip_eq_fast(vnode_bitmaps.into_iter())).enumerate() { @@ -343,7 +343,7 @@ impl LocalQueryExecution { let exchange_source = ExchangeSource { task_output_id: Some(TaskOutputId { task_id: Some(PbTaskId { - task_id: idx as u32, + task_id: idx as u64, stage_id: exchange_source_stage_id, query_id: self.query.query_id.id.clone(), }), @@ -389,7 +389,7 @@ impl LocalQueryExecution { let exchange_source = ExchangeSource { task_output_id: Some(TaskOutputId { task_id: Some(PbTaskId { - task_id: id as u32, + task_id: id as u64, stage_id: exchange_source_stage_id, query_id: self.query.query_id.id.clone(), }), @@ -429,7 +429,7 @@ impl LocalQueryExecution { let exchange_source = ExchangeSource { task_output_id: Some(TaskOutputId { task_id: Some(PbTaskId { - task_id: idx as u32, + task_id: idx as u64, stage_id: exchange_source_stage_id, query_id: self.query.query_id.id.clone(), }), @@ -531,7 +531,8 @@ impl LocalQueryExecution { )?; // TODO: should we use `pb::ParallelUnitMapping` here? - node.inner_side_vnode_mapping = mapping.to_expanded(); + node.inner_side_vnode_mapping = + mapping.to_expanded().into_iter().map(u64::from).collect(); node.worker_nodes = self.worker_node_manager.manager.list_worker_nodes(); } _ => unreachable!(), @@ -586,7 +587,7 @@ impl LocalQueryExecution { fn get_table_dml_vnode_mapping( &self, table_id: &TableId, - ) -> SchedulerResult { + ) -> SchedulerResult { let guard = self.front_env.catalog_reader().read_guard(); let table = guard @@ -610,11 +611,11 @@ impl LocalQueryExecution { // dml should use streaming vnode mapping let vnode_mapping = self.get_table_dml_vnode_mapping(table_id)?; let worker_node = { - let parallel_unit_ids = vnode_mapping.iter_unique().collect_vec(); + let worker_ids = vnode_mapping.iter_unique().collect_vec(); let candidates = self .worker_node_manager .manager - .get_workers_by_parallel_unit_ids(¶llel_unit_ids)?; + .get_workers_by_worker_slot_ids(&worker_ids)?; if candidates.is_empty() { return Err(BatchError::EmptyWorkerNodes.into()); } diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index 06237c5756fba..74ade3fdab836 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -30,7 +30,7 @@ use risingwave_common::bail; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::TableDesc; use risingwave_common::hash::table_distribution::TableDistribution; -use risingwave_common::hash::{ParallelUnitId, ParallelUnitMapping, VirtualNode}; +use risingwave_common::hash::{VirtualNode, WorkerSlotId, WorkerSlotMapping}; use risingwave_common::util::scan_range::ScanRange; use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; use risingwave_connector::source::filesystem::opendal_source::{OpendalGcs, OpendalS3}; @@ -75,10 +75,10 @@ impl std::fmt::Display for QueryId { pub type StageId = u32; // Root stage always has only one task. -pub const ROOT_TASK_ID: u32 = 0; +pub const ROOT_TASK_ID: u64 = 0; // Root task has only one output. -pub const ROOT_TASK_OUTPUT_ID: u32 = 0; -pub type TaskId = u32; +pub const ROOT_TASK_OUTPUT_ID: u64 = 0; +pub type TaskId = u64; /// Generated by [`BatchPlanFragmenter`] and used in query execution graph. #[derive(Clone, Debug)] @@ -392,12 +392,12 @@ pub struct TableScanInfo { /// full vnode bitmap, since we need to know where to schedule the singleton scan task. /// /// `None` iff the table is a system table. - partitions: Option>, + partitions: Option>, } impl TableScanInfo { /// For normal tables, `partitions` should always be `Some`. - pub fn new(name: String, partitions: HashMap) -> Self { + pub fn new(name: String, partitions: HashMap) -> Self { Self { name, partitions: Some(partitions), @@ -416,7 +416,7 @@ impl TableScanInfo { self.name.as_ref() } - pub fn partitions(&self) -> Option<&HashMap> { + pub fn partitions(&self) -> Option<&HashMap> { self.partitions.as_ref() } } @@ -1158,10 +1158,10 @@ impl BatchPlanFragmenter { fn derive_partitions( scan_ranges: &[ScanRange], table_desc: &TableDesc, - vnode_mapping: &ParallelUnitMapping, -) -> SchedulerResult> { + vnode_mapping: &WorkerSlotMapping, +) -> SchedulerResult> { let num_vnodes = vnode_mapping.len(); - let mut partitions: HashMap)> = HashMap::new(); + let mut partitions: HashMap)> = HashMap::new(); if scan_ranges.is_empty() { return Ok(vnode_mapping @@ -1190,9 +1190,9 @@ fn derive_partitions( None => { // put this scan_range to all partitions vnode_mapping.to_bitmaps().into_iter().for_each( - |(parallel_unit_id, vnode_bitmap)| { + |(worker_slot_id, vnode_bitmap)| { let (bitmap, scan_ranges) = partitions - .entry(parallel_unit_id) + .entry(worker_slot_id) .or_insert_with(|| (BitmapBuilder::zeroed(num_vnodes), vec![])); vnode_bitmap .iter() @@ -1204,9 +1204,9 @@ fn derive_partitions( } // scan a single partition Some(vnode) => { - let parallel_unit_id = vnode_mapping[vnode]; + let worker_slot_id = vnode_mapping[vnode]; let (bitmap, scan_ranges) = partitions - .entry(parallel_unit_id) + .entry(worker_slot_id) .or_insert_with(|| (BitmapBuilder::zeroed(num_vnodes), vec![])); bitmap.set(vnode.to_index(), true); scan_ranges.push(scan_range.to_protobuf()); diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index e9a5e4a017ad4..f3fec987c2bf1 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -24,7 +24,7 @@ use risingwave_pb::hummock::WriteLimits; use risingwave_pb::meta::meta_snapshot::SnapshotVersion; use risingwave_pb::meta::notification_service_server::NotificationService; use risingwave_pb::meta::{ - FragmentParallelUnitMapping, GetSessionParamsResponse, MetaSnapshot, SubscribeRequest, + FragmentWorkerSlotMapping, GetSessionParamsResponse, MetaSnapshot, SubscribeRequest, SubscribeType, }; use risingwave_pb::user::UserInfo; @@ -138,9 +138,9 @@ impl NotificationServiceImpl { } } - async fn get_parallel_unit_mapping_snapshot( + async fn get_worker_slot_mapping_snapshot( &self, - ) -> MetaResult<(Vec, NotificationVersion)> { + ) -> MetaResult<(Vec, NotificationVersion)> { match &self.metadata_manager { MetadataManager::V1(mgr) => { let fragment_guard = mgr.fragment_manager.get_fragment_read_guard().await; @@ -161,11 +161,11 @@ impl NotificationServiceImpl { } } - fn get_serving_vnode_mappings(&self) -> Vec { + fn get_serving_vnode_mappings(&self) -> Vec { self.serving_vnode_mapping .all() .iter() - .map(|(fragment_id, mapping)| FragmentParallelUnitMapping { + .map(|(fragment_id, mapping)| FragmentWorkerSlotMapping { fragment_id: *fragment_id, mapping: Some(mapping.to_protobuf()), }) @@ -241,9 +241,11 @@ impl NotificationServiceImpl { users, catalog_version, ) = self.get_catalog_snapshot().await?; - let (parallel_unit_mappings, parallel_unit_mapping_version) = - self.get_parallel_unit_mapping_snapshot().await?; - let serving_parallel_unit_mappings = self.get_serving_vnode_mappings(); + + let (streaming_worker_slot_mappings, streaming_worker_slot_mapping_version) = + self.get_worker_slot_mapping_snapshot().await?; + let serving_worker_slot_mappings = self.get_serving_vnode_mappings(); + let (nodes, worker_node_version) = self.get_worker_node_snapshot().await?; let hummock_snapshot = Some(self.hummock_manager.latest_snapshot()); @@ -270,15 +272,15 @@ impl NotificationServiceImpl { functions, connections, users, - parallel_unit_mappings, nodes, hummock_snapshot, - serving_parallel_unit_mappings, version: Some(SnapshotVersion { catalog_version, - parallel_unit_mapping_version, worker_node_version, + streaming_worker_slot_mapping_version, }), + serving_worker_slot_mappings, + streaming_worker_slot_mappings, session_params, ..Default::default() }) diff --git a/src/meta/service/src/serving_service.rs b/src/meta/service/src/serving_service.rs index d1b013e078e0f..92c5fd9db4837 100644 --- a/src/meta/service/src/serving_service.rs +++ b/src/meta/service/src/serving_service.rs @@ -16,7 +16,7 @@ use itertools::Itertools; use risingwave_meta::manager::MetadataManager; use risingwave_pb::meta::serving_service_server::ServingService; use risingwave_pb::meta::{ - FragmentParallelUnitMapping, GetServingVnodeMappingsRequest, GetServingVnodeMappingsResponse, + FragmentWorkerSlotMapping, GetServingVnodeMappingsRequest, GetServingVnodeMappingsResponse, }; use tonic::{Request, Response, Status}; @@ -49,7 +49,7 @@ impl ServingService for ServingServiceImpl { .serving_vnode_mapping .all() .into_iter() - .map(|(fragment_id, mapping)| FragmentParallelUnitMapping { + .map(|(fragment_id, mapping)| FragmentWorkerSlotMapping { fragment_id, mapping: Some(mapping.to_protobuf()), }) @@ -78,8 +78,8 @@ impl ServingService for ServingServiceImpl { } }; Ok(Response::new(GetServingVnodeMappingsResponse { - mappings, fragment_to_table, + worker_slot_mappings: mappings, })) } } diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index e6c77dfb1b1fb..cb3e386c34fc5 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -19,6 +19,7 @@ use std::sync::Arc; use anyhow::anyhow; use itertools::Itertools; use risingwave_common::catalog::{TableOption, DEFAULT_SCHEMA_NAME, SYSTEM_SCHEMAS}; +use risingwave_common::hash::ParallelUnitMapping; use risingwave_common::util::stream_graph_visitor::visit_stream_node_cont_mut; use risingwave_common::{bail, current_cluster_version}; use risingwave_connector::source::UPSTREAM_SOURCE_KEY; @@ -45,7 +46,9 @@ use risingwave_pb::meta::relation::PbRelationInfo; use risingwave_pb::meta::subscribe_response::{ Info as NotificationInfo, Info, Operation as NotificationOperation, Operation, }; -use risingwave_pb::meta::{PbRelation, PbRelationGroup}; +use risingwave_pb::meta::{ + FragmentParallelUnitMapping, PbFragmentWorkerSlotMapping, PbRelation, PbRelationGroup, +}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::FragmentTypeFlag; use risingwave_pb::user::PbUserInfo; @@ -64,9 +67,9 @@ use crate::controller::utils::{ check_connection_name_duplicate, check_database_name_duplicate, check_function_signature_duplicate, check_relation_name_duplicate, check_schema_name_duplicate, ensure_object_id, ensure_object_not_refer, ensure_schema_empty, ensure_user_id, - get_fragment_mappings_by_jobs, get_referring_objects, get_referring_objects_cascade, - get_user_privilege, list_user_info_by_ids, resolve_source_register_info_for_jobs, - PartialObject, + get_fragment_mappings_by_jobs, get_parallel_unit_to_worker_map, get_referring_objects, + get_referring_objects_cascade, get_user_privilege, list_user_info_by_ids, + resolve_source_register_info_for_jobs, PartialObject, }; use crate::controller::ObjectModel; use crate::manager::{Catalog, MetaSrvEnv, NotificationVersion, IGNORED_NOTIFICATION_VERSION}; @@ -275,8 +278,30 @@ impl CatalogController { .into_tuple() .all(&txn) .await?; + + let parallel_unit_to_worker = get_parallel_unit_to_worker_map(&txn).await?; + let fragment_mappings = get_fragment_mappings_by_jobs(&txn, streaming_jobs.clone()).await?; + let fragment_mappings = fragment_mappings + .into_iter() + .map( + |FragmentParallelUnitMapping { + fragment_id, + mapping, + }| { + PbFragmentWorkerSlotMapping { + fragment_id, + mapping: Some( + ParallelUnitMapping::from_protobuf(&mapping.unwrap()) + .to_worker_slot(¶llel_unit_to_worker) + .to_protobuf(), + ), + } + }, + ) + .collect(); + // The schema and objects in the database will be delete cascade. let res = Object::delete_by_id(database_id).exec(&txn).await?; if res.rows_affected == 0 { @@ -296,6 +321,7 @@ impl CatalogController { }), ) .await; + self.notify_fragment_mapping(NotificationOperation::Delete, fragment_mappings) .await; Ok(( @@ -1973,6 +1999,7 @@ impl CatalogController { let (source_fragments, removed_actors) = resolve_source_register_info_for_jobs(&txn, to_drop_streaming_jobs.clone()).await?; + let fragment_mappings = get_fragment_mappings_by_jobs(&txn, to_drop_streaming_jobs.clone()).await?; @@ -1999,6 +2026,8 @@ impl CatalogController { } let user_infos = list_user_info_by_ids(to_update_user_ids, &txn).await?; + let parallel_unit_to_worker = get_parallel_unit_to_worker_map(&txn).await?; + txn.commit().await?; // notify about them. @@ -2073,6 +2102,26 @@ impl CatalogController { NotificationInfo::RelationGroup(PbRelationGroup { relations }), ) .await; + + let fragment_mappings = fragment_mappings + .into_iter() + .map( + |FragmentParallelUnitMapping { + fragment_id, + mapping, + }| { + PbFragmentWorkerSlotMapping { + fragment_id, + mapping: Some( + ParallelUnitMapping::from_protobuf(&mapping.unwrap()) + .to_worker_slot(¶llel_unit_to_worker) + .to_protobuf(), + ), + } + }, + ) + .collect(); + self.notify_fragment_mapping(NotificationOperation::Delete, fragment_mappings) .await; diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 7bf5b02f971af..fa53773741484 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -18,6 +18,7 @@ use std::mem::swap; use anyhow::Context; use itertools::Itertools; use risingwave_common::bail; +use risingwave_common::hash::ParallelUnitMapping; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_meta_model_v2::actor::ActorStatus; use risingwave_meta_model_v2::prelude::{Actor, ActorDispatcher, Fragment, Sink, StreamingJob}; @@ -34,7 +35,7 @@ use risingwave_pb::meta::table_fragments::actor_status::PbActorState; use risingwave_pb::meta::table_fragments::fragment::PbFragmentDistributionType; use risingwave_pb::meta::table_fragments::{PbActorStatus, PbFragment, PbState}; use risingwave_pb::meta::{ - FragmentParallelUnitMapping, PbFragmentParallelUnitMapping, PbTableFragments, + FragmentWorkerSlotMapping, PbFragmentWorkerSlotMapping, PbTableFragments, }; use risingwave_pb::source::PbConnectorSplits; use risingwave_pb::stream_plan::stream_node::NodeBody; @@ -50,7 +51,8 @@ use sea_orm::{ use crate::controller::catalog::{CatalogController, CatalogControllerInner}; use crate::controller::utils::{ - get_actor_dispatchers, FragmentDesc, PartialActorLocation, PartialFragmentStateTables, + get_actor_dispatchers, get_parallel_unit_to_worker_map, FragmentDesc, PartialActorLocation, + PartialFragmentStateTables, }; use crate::manager::{ActorInfos, LocalNotification}; use crate::model::TableParallelism; @@ -61,7 +63,9 @@ impl CatalogControllerInner { /// List all fragment vnode mapping info for all CREATED streaming jobs. pub async fn all_running_fragment_mappings( &self, - ) -> MetaResult + '_> { + ) -> MetaResult + '_> { + let txn = self.db.begin().await?; + let fragment_mappings: Vec<(FragmentId, FragmentVnodeMapping)> = Fragment::find() .join(JoinType::InnerJoin, fragment::Relation::Object.def()) .join(JoinType::InnerJoin, object::Relation::StreamingJob.def()) @@ -69,14 +73,24 @@ impl CatalogControllerInner { .columns([fragment::Column::FragmentId, fragment::Column::VnodeMapping]) .filter(streaming_job::Column::JobStatus.eq(JobStatus::Created)) .into_tuple() - .all(&self.db) + .all(&txn) .await?; - Ok(fragment_mappings.into_iter().map(|(fragment_id, mapping)| { - FragmentParallelUnitMapping { - fragment_id: fragment_id as _, - mapping: Some(mapping.to_protobuf()), - } - })) + + let parallel_unit_to_worker = get_parallel_unit_to_worker_map(&txn).await?; + + Ok(fragment_mappings + .into_iter() + .map(move |(fragment_id, mapping)| { + let worker_slot_mapping = + ParallelUnitMapping::from_protobuf(&mapping.to_protobuf()) + .to_worker_slot(¶llel_unit_to_worker) + .to_protobuf(); + + FragmentWorkerSlotMapping { + fragment_id: fragment_id as _, + mapping: Some(worker_slot_mapping), + } + })) } } @@ -84,7 +98,7 @@ impl CatalogController { pub(crate) async fn notify_fragment_mapping( &self, operation: NotificationOperation, - fragment_mappings: Vec, + fragment_mappings: Vec, ) { let fragment_ids = fragment_mappings .iter() @@ -96,7 +110,7 @@ impl CatalogController { .notification_manager() .notify_frontend( operation, - NotificationInfo::ParallelUnitMapping(fragment_mapping), + NotificationInfo::StreamingWorkerSlotMapping(fragment_mapping), ) .await; } @@ -936,15 +950,21 @@ impl CatalogController { .await?; } + let parallel_unit_to_worker = get_parallel_unit_to_worker_map(&txn).await?; + txn.commit().await?; self.notify_fragment_mapping( NotificationOperation::Update, fragment_mapping .into_iter() - .map(|(fragment_id, mapping)| PbFragmentParallelUnitMapping { + .map(|(fragment_id, mapping)| PbFragmentWorkerSlotMapping { fragment_id: fragment_id as _, - mapping: Some(mapping.to_protobuf()), + mapping: Some( + ParallelUnitMapping::from_protobuf(&mapping.to_protobuf()) + .to_worker_slot(¶llel_unit_to_worker) + .to_protobuf(), + ), }) .collect(), ) diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index db9c1d03eab4b..8c597041e2f75 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -43,7 +43,7 @@ use risingwave_pb::meta::subscribe_response::{ }; use risingwave_pb::meta::table_fragments::PbActorStatus; use risingwave_pb::meta::{ - FragmentParallelUnitMapping, PbFragmentParallelUnitMapping, PbRelation, PbRelationGroup, + FragmentWorkerSlotMapping, PbFragmentWorkerSlotMapping, PbRelation, PbRelationGroup, PbTableFragments, Relation, }; use risingwave_pb::source::{PbConnectorSplit, PbConnectorSplits}; @@ -66,7 +66,7 @@ use crate::controller::catalog::CatalogController; use crate::controller::rename::ReplaceTableExprRewriter; use crate::controller::utils::{ check_relation_name_duplicate, check_sink_into_table_cycle, ensure_object_id, ensure_user_id, - get_fragment_actor_ids, get_fragment_mappings, + get_fragment_actor_ids, get_fragment_mappings, get_parallel_unit_to_worker_map, }; use crate::controller::ObjectModel; use crate::manager::{NotificationVersion, SinkId, StreamingJob}; @@ -803,7 +803,7 @@ impl CatalogController { dropping_sink_id: Option, txn: &DatabaseTransaction, streaming_job: StreamingJob, - ) -> MetaResult<(Vec, Vec)> { + ) -> MetaResult<(Vec, Vec)> { // Question: The source catalog should be remain unchanged? let StreamingJob::Table(_, table, ..) = streaming_job else { unreachable!("unexpected job: {streaming_job:?}") @@ -1007,8 +1007,7 @@ impl CatalogController { } } - let fragment_mapping: Vec = - get_fragment_mappings(txn, job_id as _).await?; + let fragment_mapping: Vec<_> = get_fragment_mappings(txn, job_id as _).await?; Ok((relations, fragment_mapping)) } @@ -1224,6 +1223,8 @@ impl CatalogController { let txn = inner.db.begin().await?; + let parallel_unit_to_worker = get_parallel_unit_to_worker_map(&txn).await?; + let mut fragment_mapping_to_notify = vec![]; // for assert only @@ -1399,9 +1400,13 @@ impl CatalogController { fragment.vnode_mapping = Set((&vnode_mapping).into()); fragment.update(&txn).await?; - fragment_mapping_to_notify.push(FragmentParallelUnitMapping { + let worker_slot_mapping = ParallelUnitMapping::from_protobuf(&vnode_mapping) + .to_worker_slot(¶llel_unit_to_worker) + .to_protobuf(); + + fragment_mapping_to_notify.push(FragmentWorkerSlotMapping { fragment_id: fragment_id as u32, - mapping: Some(vnode_mapping), + mapping: Some(worker_slot_mapping), }); // for downstream and upstream diff --git a/src/meta/src/controller/utils.rs b/src/meta/src/controller/utils.rs index 7d8f4769e8264..b654050708546 100644 --- a/src/meta/src/controller/utils.rs +++ b/src/meta/src/controller/utils.rs @@ -16,6 +16,7 @@ use std::collections::{BTreeSet, HashMap, HashSet}; use anyhow::anyhow; use itertools::Itertools; +use risingwave_common::hash::ParallelUnitMapping; use risingwave_meta_model_migration::WithQuery; use risingwave_meta_model_v2::actor::ActorStatus; use risingwave_meta_model_v2::fragment::DistributionType; @@ -24,11 +25,11 @@ use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::{ actor, actor_dispatcher, connection, database, fragment, function, index, object, object_dependency, schema, sink, source, subscription, table, user, user_privilege, view, - ActorId, DataTypeArray, DatabaseId, FragmentId, FragmentVnodeMapping, I32Array, ObjectId, - PrivilegeId, SchemaId, SourceId, StreamNode, UserId, + worker_property, ActorId, DataTypeArray, DatabaseId, FragmentId, FragmentVnodeMapping, + I32Array, ObjectId, PrivilegeId, SchemaId, SourceId, StreamNode, UserId, WorkerId, }; use risingwave_pb::catalog::{PbConnection, PbFunction, PbSubscription}; -use risingwave_pb::meta::PbFragmentParallelUnitMapping; +use risingwave_pb::meta::{PbFragmentParallelUnitMapping, PbFragmentWorkerSlotMapping}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{PbFragmentTypeFlag, PbStreamNode, StreamSource}; use risingwave_pb::user::grant_privilege::{PbAction, PbActionWithGrantOption, PbObject}; @@ -815,10 +816,12 @@ where pub async fn get_fragment_mappings( db: &C, job_id: ObjectId, -) -> MetaResult> +) -> MetaResult> where C: ConnectionTrait, { + let parallel_unit_to_worker = get_parallel_unit_to_worker_map(db).await?; + let fragment_mappings: Vec<(FragmentId, FragmentVnodeMapping)> = Fragment::find() .select_only() .columns([fragment::Column::FragmentId, fragment::Column::VnodeMapping]) @@ -829,9 +832,13 @@ where Ok(fragment_mappings .into_iter() - .map(|(fragment_id, mapping)| PbFragmentParallelUnitMapping { + .map(|(fragment_id, mapping)| PbFragmentWorkerSlotMapping { fragment_id: fragment_id as _, - mapping: Some(mapping.to_protobuf()), + mapping: Some( + ParallelUnitMapping::from_protobuf(&mapping.to_protobuf()) + .to_worker_slot(¶llel_unit_to_worker) + .to_protobuf(), + ), }) .collect()) } @@ -948,3 +955,30 @@ where Ok((source_fragment_ids, actors.into_iter().collect())) } + +pub(crate) async fn get_parallel_unit_to_worker_map(db: &C) -> MetaResult> +where + C: ConnectionTrait, +{ + let worker_parallel_units = WorkerProperty::find() + .select_only() + .columns([ + worker_property::Column::WorkerId, + worker_property::Column::ParallelUnitIds, + ]) + .into_tuple::<(WorkerId, I32Array)>() + .all(db) + .await?; + + let parallel_unit_to_worker = worker_parallel_units + .into_iter() + .flat_map(|(worker_id, parallel_unit_ids)| { + parallel_unit_ids + .into_inner() + .into_iter() + .map(move |parallel_unit_id| (parallel_unit_id as u32, worker_id as u32)) + }) + .collect::>(); + + Ok(parallel_unit_to_worker) +} diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index 92c2d11c01b5f..9cb0d25a09ae0 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -26,10 +26,11 @@ use risingwave_common::util::stream_graph_visitor::{ }; use risingwave_connector::source::SplitImpl; use risingwave_meta_model_v2::SourceId; +use risingwave_pb::common::{PbParallelUnitMapping, PbWorkerSlotMapping}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::table_fragments::actor_status::ActorState; use risingwave_pb::meta::table_fragments::{ActorStatus, Fragment, State}; -use risingwave_pb::meta::FragmentParallelUnitMapping; +use risingwave_pb::meta::FragmentWorkerSlotMapping; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::update_mutation::MergeUpdate; use risingwave_pb::stream_plan::{ @@ -58,18 +59,21 @@ impl FragmentManagerCore { /// List all fragment vnode mapping info that not in `State::Initial`. pub fn all_running_fragment_mappings( &self, - ) -> impl Iterator + '_ { + ) -> impl Iterator + '_ { self.table_fragments .values() .filter(|tf| tf.state() != State::Initial) .flat_map(|table_fragments| { - table_fragments.fragments.values().map(|fragment| { - let parallel_unit_mapping = fragment.vnode_mapping.clone().unwrap(); - FragmentParallelUnitMapping { + table_fragments + .fragments + .values() + .map(move |fragment| FragmentWorkerSlotMapping { fragment_id: fragment.fragment_id, - mapping: Some(parallel_unit_mapping), - } - }) + mapping: Some(FragmentManager::convert_mapping( + &table_fragments.actor_status, + fragment.vnode_mapping.as_ref().unwrap(), + )), + }) }) } @@ -194,18 +198,23 @@ impl FragmentManager { async fn notify_fragment_mapping(&self, table_fragment: &TableFragments, operation: Operation) { // Notify all fragment mapping to frontend nodes for fragment in table_fragment.fragments.values() { - let mapping = fragment - .vnode_mapping - .clone() - .expect("no data distribution found"); - let fragment_mapping = FragmentParallelUnitMapping { + let fragment_mapping = FragmentWorkerSlotMapping { fragment_id: fragment.fragment_id, - mapping: Some(mapping), + mapping: Some(Self::convert_mapping( + &table_fragment.actor_status, + fragment + .vnode_mapping + .as_ref() + .expect("no data distribution found"), + )), }; self.env .notification_manager() - .notify_frontend(operation, Info::ParallelUnitMapping(fragment_mapping)) + .notify_frontend( + operation, + Info::StreamingWorkerSlotMapping(fragment_mapping), + ) .await; } @@ -1264,11 +1273,14 @@ impl FragmentManager { *fragment.vnode_mapping.as_mut().unwrap() = vnode_mapping.clone(); + let worker_slot_mapping = Self::convert_mapping(&actor_status, &vnode_mapping); + // Notify fragment mapping to frontend nodes. - let fragment_mapping = FragmentParallelUnitMapping { + let fragment_mapping = FragmentWorkerSlotMapping { fragment_id: *fragment_id as FragmentId, - mapping: Some(vnode_mapping), + mapping: Some(worker_slot_mapping), }; + fragment_mapping_to_notify.push(fragment_mapping); } @@ -1388,13 +1400,30 @@ impl FragmentManager { for mapping in fragment_mapping_to_notify { self.env .notification_manager() - .notify_frontend(Operation::Update, Info::ParallelUnitMapping(mapping)) + .notify_frontend(Operation::Update, Info::StreamingWorkerSlotMapping(mapping)) .await; } Ok(()) } + fn convert_mapping( + actor_status: &BTreeMap, + vnode_mapping: &PbParallelUnitMapping, + ) -> PbWorkerSlotMapping { + let parallel_unit_to_worker = actor_status + .values() + .map(|actor_status| { + let parallel_unit = actor_status.get_parallel_unit().unwrap(); + (parallel_unit.id, parallel_unit.worker_node_id) + }) + .collect(); + + ParallelUnitMapping::from_protobuf(vnode_mapping) + .to_worker_slot(¶llel_unit_to_worker) + .to_protobuf() + } + pub async fn table_node_actors( &self, table_ids: &HashSet, diff --git a/src/meta/src/serving/mod.rs b/src/meta/src/serving/mod.rs index 36e7b77ccf63a..69e17a978212e 100644 --- a/src/meta/src/serving/mod.rs +++ b/src/meta/src/serving/mod.rs @@ -16,11 +16,11 @@ use std::collections::HashMap; use std::sync::Arc; use parking_lot::RwLock; -use risingwave_common::hash::ParallelUnitMapping; +use risingwave_common::hash::WorkerSlotMapping; use risingwave_common::vnode_mapping::vnode_placement::place_vnode; use risingwave_pb::common::{WorkerNode, WorkerType}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; -use risingwave_pb::meta::{FragmentParallelUnitMapping, FragmentParallelUnitMappings}; +use risingwave_pb::meta::{FragmentWorkerSlotMapping, FragmentWorkerSlotMappings}; use tokio::sync::oneshot::Sender; use tokio::task::JoinHandle; @@ -31,11 +31,11 @@ pub type ServingVnodeMappingRef = Arc; #[derive(Default)] pub struct ServingVnodeMapping { - serving_vnode_mappings: RwLock>, + serving_vnode_mappings: RwLock>, } impl ServingVnodeMapping { - pub fn all(&self) -> HashMap { + pub fn all(&self) -> HashMap { self.serving_vnode_mappings.read().clone() } @@ -45,9 +45,9 @@ impl ServingVnodeMapping { &self, streaming_parallelisms: HashMap, workers: &[WorkerNode], - ) -> (HashMap, Vec) { + ) -> (HashMap, Vec) { let mut serving_vnode_mappings = self.serving_vnode_mappings.write(); - let mut upserted: HashMap = HashMap::default(); + let mut upserted: HashMap = HashMap::default(); let mut failed: Vec = vec![]; for (fragment_id, streaming_parallelism) in streaming_parallelisms { let new_mapping = { @@ -81,24 +81,24 @@ impl ServingVnodeMapping { } } -pub(crate) fn to_fragment_parallel_unit_mapping( - mappings: &HashMap, -) -> Vec { +pub(crate) fn to_fragment_worker_slot_mapping( + mappings: &HashMap, +) -> Vec { mappings .iter() - .map(|(fragment_id, mapping)| FragmentParallelUnitMapping { + .map(|(fragment_id, mapping)| FragmentWorkerSlotMapping { fragment_id: *fragment_id, mapping: Some(mapping.to_protobuf()), }) .collect() } -pub(crate) fn to_deleted_fragment_parallel_unit_mapping( +pub(crate) fn to_deleted_fragment_worker_slot_mapping( fragment_ids: &[FragmentId], -) -> Vec { +) -> Vec { fragment_ids .iter() - .map(|fragment_id| FragmentParallelUnitMapping { + .map(|fragment_id| FragmentWorkerSlotMapping { fragment_id: *fragment_id, mapping: None, }) @@ -120,8 +120,8 @@ pub async fn on_meta_start( ); notification_manager.notify_frontend_without_version( Operation::Snapshot, - Info::ServingParallelUnitMappings(FragmentParallelUnitMappings { - mappings: to_fragment_parallel_unit_mapping(&mappings), + Info::ServingWorkerSlotMappings(FragmentWorkerSlotMappings { + mappings: to_fragment_worker_slot_mapping(&mappings), }), ); } @@ -185,7 +185,7 @@ pub async fn start_serving_vnode_mapping_worker( let (workers, streaming_parallelisms) = fetch_serving_infos(&metadata_manager).await; let (mappings, _) = serving_vnode_mapping.upsert(streaming_parallelisms, &workers); tracing::debug!("Update serving vnode mapping snapshot for fragments {:?}.", mappings.keys()); - notification_manager.notify_frontend_without_version(Operation::Snapshot, Info::ServingParallelUnitMappings(FragmentParallelUnitMappings{ mappings: to_fragment_parallel_unit_mapping(&mappings) })); + notification_manager.notify_frontend_without_version(Operation::Snapshot, Info::ServingWorkerSlotMappings(FragmentWorkerSlotMappings{ mappings: to_fragment_worker_slot_mapping(&mappings) })); } LocalNotification::FragmentMappingsUpsert(fragment_ids) => { if fragment_ids.is_empty() { @@ -195,11 +195,11 @@ pub async fn start_serving_vnode_mapping_worker( let (upserted, failed) = serving_vnode_mapping.upsert(streaming_parallelisms, &workers); if !upserted.is_empty() { tracing::debug!("Update serving vnode mapping for fragments {:?}.", upserted.keys()); - notification_manager.notify_frontend_without_version(Operation::Update, Info::ServingParallelUnitMappings(FragmentParallelUnitMappings{ mappings: to_fragment_parallel_unit_mapping(&upserted) })); + notification_manager.notify_frontend_without_version(Operation::Update, Info::ServingWorkerSlotMappings(FragmentWorkerSlotMappings{ mappings: to_fragment_worker_slot_mapping(&upserted) })); } if !failed.is_empty() { tracing::debug!("Fail to update serving vnode mapping for fragments {:?}.", failed); - notification_manager.notify_frontend_without_version(Operation::Delete, Info::ServingParallelUnitMappings(FragmentParallelUnitMappings{ mappings: to_deleted_fragment_parallel_unit_mapping(&failed)})); + notification_manager.notify_frontend_without_version(Operation::Delete, Info::ServingWorkerSlotMappings(FragmentWorkerSlotMappings{ mappings: to_deleted_fragment_worker_slot_mapping(&failed)})); } } LocalNotification::FragmentMappingsDelete(fragment_ids) => { @@ -208,7 +208,7 @@ pub async fn start_serving_vnode_mapping_worker( } tracing::debug!("Delete serving vnode mapping for fragments {:?}.", fragment_ids); serving_vnode_mapping.remove(&fragment_ids); - notification_manager.notify_frontend_without_version(Operation::Delete, Info::ServingParallelUnitMappings(FragmentParallelUnitMappings{ mappings: to_deleted_fragment_parallel_unit_mapping(&fragment_ids) })); + notification_manager.notify_frontend_without_version(Operation::Delete, Info::ServingWorkerSlotMappings(FragmentWorkerSlotMappings{ mappings: to_deleted_fragment_worker_slot_mapping(&fragment_ids) })); } _ => {} } diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index 8c673b4dc2409..34759c09ab720 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -41,7 +41,7 @@ use risingwave_pb::meta::table_fragments::fragment::{ FragmentDistributionType, PbFragmentDistributionType, }; use risingwave_pb::meta::table_fragments::{self, ActorStatus, PbFragment, State}; -use risingwave_pb::meta::FragmentParallelUnitMappings; +use risingwave_pb::meta::FragmentWorkerSlotMappings; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ Dispatcher, DispatcherType, FragmentTypeFlag, PbStreamActor, StreamNode, @@ -61,8 +61,7 @@ use crate::manager::{ }; use crate::model::{ActorId, DispatcherId, FragmentId, TableFragments, TableParallelism}; use crate::serving::{ - to_deleted_fragment_parallel_unit_mapping, to_fragment_parallel_unit_mapping, - ServingVnodeMapping, + to_deleted_fragment_worker_slot_mapping, to_fragment_worker_slot_mapping, ServingVnodeMapping, }; use crate::storage::{MetaStore, MetaStoreError, MetaStoreRef, Transaction, DEFAULT_COLUMN_FAMILY}; use crate::stream::{GlobalStreamManager, SourceManagerRef}; @@ -1725,8 +1724,8 @@ impl ScaleController { .notification_manager() .notify_frontend_without_version( Operation::Update, - Info::ServingParallelUnitMappings(FragmentParallelUnitMappings { - mappings: to_fragment_parallel_unit_mapping(&upserted), + Info::ServingWorkerSlotMappings(FragmentWorkerSlotMappings { + mappings: to_fragment_worker_slot_mapping(&upserted), }), ); } @@ -1739,8 +1738,8 @@ impl ScaleController { .notification_manager() .notify_frontend_without_version( Operation::Delete, - Info::ServingParallelUnitMappings(FragmentParallelUnitMappings { - mappings: to_deleted_fragment_parallel_unit_mapping(&failed), + Info::ServingWorkerSlotMappings(FragmentWorkerSlotMappings { + mappings: to_deleted_fragment_worker_slot_mapping(&failed), }), ); } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 7fd32f3b8bab0..e4d6b53003788 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -25,7 +25,7 @@ use futures::stream::BoxStream; use lru::LruCache; use risingwave_common::catalog::{CatalogVersion, FunctionId, IndexId, TableId}; use risingwave_common::config::{MetaConfig, MAX_CONNECTION_WINDOW_SIZE}; -use risingwave_common::hash::ParallelUnitMapping; +use risingwave_common::hash::WorkerSlotMapping; use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::telemetry::report::TelemetryInfoFetcher; use risingwave_common::util::addr::HostAddr; @@ -1171,11 +1171,11 @@ impl MetaClient { pub async fn list_serving_vnode_mappings( &self, - ) -> Result> { + ) -> Result> { let req = GetServingVnodeMappingsRequest {}; let resp = self.inner.get_serving_vnode_mappings(req).await?; let mappings = resp - .mappings + .worker_slot_mappings .into_iter() .map(|p| { ( @@ -1185,7 +1185,7 @@ impl MetaClient { .get(&p.fragment_id) .cloned() .unwrap_or(0), - ParallelUnitMapping::from_protobuf(p.mapping.as_ref().unwrap()), + WorkerSlotMapping::from_protobuf(p.mapping.as_ref().unwrap()), ), ) }) From 469c38060724d8dcc20533709a7c4dee97079c86 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Tue, 28 May 2024 15:18:57 +0800 Subject: [PATCH 04/13] fix(object store): do not call abort when streaming upload finish error and fix azure workload identity (#16961) --- Cargo.lock | 2 +- Cargo.toml | 2 +- .../src/object/opendal_engine/opendal_object_store.rs | 10 +++++----- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b398285c98948..4b27d8901dc36 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9747,7 +9747,7 @@ dependencies = [ [[package]] name = "reqsign" version = "0.14.9" -source = "git+https://github.com/wcy-fdu/reqsign.git?rev=002ee2a#002ee2a41749b08bb5336f344e31f514d8fce718" +source = "git+https://github.com/wcy-fdu/reqsign.git?rev=c7dd668#c7dd668764ada1e7477177cfa913fec24252dd34" dependencies = [ "anyhow", "async-trait", diff --git a/Cargo.toml b/Cargo.toml index a755e705277bc..96f647a634f38 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -317,7 +317,7 @@ futures-timer = { git = "https://github.com/madsim-rs/futures-timer.git", rev = # patch: unlimit 4MB message size for grpc client etcd-client = { git = "https://github.com/risingwavelabs/etcd-client.git", rev = "4e84d40" } # todo(wcy-fdu): remove this patch fork after opendal release a new version to apply azure workload identity change. -reqsign = { git = "https://github.com/wcy-fdu/reqsign.git", rev = "002ee2a" } +reqsign = { git = "https://github.com/wcy-fdu/reqsign.git", rev = "c7dd668" } # patch to remove preserve_order from serde_json deno_core = { git = "https://github.com/bakjos/deno_core", rev = "9b241c6" } # patch to user reqwest 0.12.2 diff --git a/src/object_store/src/object/opendal_engine/opendal_object_store.rs b/src/object_store/src/object/opendal_engine/opendal_object_store.rs index 13229a110e754..47ca4f362702a 100644 --- a/src/object_store/src/object/opendal_engine/opendal_object_store.rs +++ b/src/object_store/src/object/opendal_engine/opendal_object_store.rs @@ -248,10 +248,7 @@ impl ObjectStore for OpendalObjectStore { } fn support_streaming_upload(&self) -> bool { - match self.engine_type { - EngineType::Azblob => false, - _ => self.op.info().native_capability().write_can_multi, - } + self.op.info().native_capability().write_can_multi } } @@ -300,7 +297,10 @@ impl StreamingUploader for OpendalStreamingUploader { match self.writer.close().await { Ok(_) => (), Err(err) => { - self.writer.abort().await?; + // Due to a bug in OpenDAL, calling `abort()` here may trigger unreachable code and cause panic. + // refer to https://github.com/apache/opendal/issues/4651 + // This will be fixed after the next bump in the opendal version. + // self.writer.abort().await?; return Err(err.into()); } }; From 2b525362d3fd39e19eabae52c381842df8c7f6e0 Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 28 May 2024 15:57:47 +0800 Subject: [PATCH 05/13] feat(source): parse avro map into jsonb (#16948) Signed-off-by: xxchan --- Cargo.lock | 1 + src/connector/Cargo.toml | 1 + src/connector/src/parser/avro/parser.rs | 43 +++-- src/connector/src/parser/avro/util.rs | 173 +++++++++++++++++- .../src/parser/debezium/avro_parser.rs | 23 ++- .../src/parser/debezium/simd_json_parser.rs | 2 +- src/connector/src/parser/json_parser.rs | 3 +- src/connector/src/parser/mod.rs | 32 ++++ src/connector/src/parser/unified/avro.rs | 21 ++- src/connector/src/parser/unified/mod.rs | 2 +- src/frontend/src/handler/create_source.rs | 1 + 11 files changed, 258 insertions(+), 44 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4b27d8901dc36..1f6bacca0fc81 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10541,6 +10541,7 @@ dependencies = [ "indexmap 1.9.3", "itertools 0.12.1", "jni", + "jsonbb", "jsonschema-transpiler", "jsonwebtoken", "madsim-rdkafka", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 849f626b22d3f..9afb1a69d0dcc 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -70,6 +70,7 @@ icelake = { workspace = true } indexmap = { version = "1.9.3", features = ["serde"] } itertools = { workspace = true } jni = { version = "0.21.1", features = ["invocation"] } +jsonbb = { workspace = true } jsonwebtoken = "9.2.0" jst = { package = 'jsonschema-transpiler', git = "https://github.com/mozilla/jsonschema-transpiler", rev = "c1a89d720d118843d8bcca51084deb0ed223e4b4" } maplit = "1.0.2" diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 540592ff383e8..7f498a055ac7e 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -27,7 +27,7 @@ use crate::error::ConnectorResult; use crate::parser::unified::avro::{AvroAccess, AvroParseOptions}; use crate::parser::unified::AccessImpl; use crate::parser::util::bytes_from_url; -use crate::parser::{AccessBuilder, EncodingProperties, EncodingType}; +use crate::parser::{AccessBuilder, AvroProperties, EncodingProperties, EncodingType, MapHandling}; use crate::schema::schema_registry::{ extract_schema_id, get_subject_by_strategy, handle_sr_list, Client, }; @@ -101,35 +101,46 @@ pub struct AvroParserConfig { pub schema: Arc, pub key_schema: Option>, pub schema_resolver: Option>, + + pub map_handling: Option, } impl AvroParserConfig { pub async fn new(encoding_properties: EncodingProperties) -> ConnectorResult { - let avro_config = try_match_expand!(encoding_properties, EncodingProperties::Avro)?; - let schema_location = &avro_config.row_schema_location; - let enable_upsert = avro_config.enable_upsert; + let AvroProperties { + use_schema_registry, + row_schema_location: schema_location, + client_config, + aws_auth_props, + topic, + enable_upsert, + record_name, + key_record_name, + name_strategy, + map_handling, + } = try_match_expand!(encoding_properties, EncodingProperties::Avro)?; let url = handle_sr_list(schema_location.as_str())?; - if avro_config.use_schema_registry { - let client = Client::new(url, &avro_config.client_config)?; + if use_schema_registry { + let client = Client::new(url, &client_config)?; let resolver = ConfluentSchemaResolver::new(client); let subject_key = if enable_upsert { Some(get_subject_by_strategy( - &avro_config.name_strategy, - avro_config.topic.as_str(), - avro_config.key_record_name.as_deref(), + &name_strategy, + topic.as_str(), + key_record_name.as_deref(), true, )?) } else { - if let Some(name) = &avro_config.key_record_name { + if let Some(name) = &key_record_name { bail!("key.message = {name} not used"); } None }; let subject_value = get_subject_by_strategy( - &avro_config.name_strategy, - avro_config.topic.as_str(), - avro_config.record_name.as_deref(), + &name_strategy, + topic.as_str(), + record_name.as_deref(), false, )?; tracing::debug!("infer key subject {subject_key:?}, value subject {subject_value}"); @@ -142,25 +153,27 @@ impl AvroParserConfig { None }, schema_resolver: Some(Arc::new(resolver)), + map_handling, }) } else { if enable_upsert { bail!("avro upsert without schema registry is not supported"); } let url = url.first().unwrap(); - let schema_content = bytes_from_url(url, avro_config.aws_auth_props.as_ref()).await?; + let schema_content = bytes_from_url(url, aws_auth_props.as_ref()).await?; let schema = Schema::parse_reader(&mut schema_content.as_slice()) .context("failed to parse avro schema")?; Ok(Self { schema: Arc::new(schema), key_schema: None, schema_resolver: None, + map_handling, }) } } pub fn map_to_columns(&self) -> ConnectorResult> { - avro_schema_to_column_descs(self.schema.as_ref()) + avro_schema_to_column_descs(self.schema.as_ref(), self.map_handling) } } diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index 9f4992060decf..4f36b15e5ce76 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -15,6 +15,7 @@ use std::sync::LazyLock; use apache_avro::schema::{DecimalSchema, RecordSchema, Schema}; +use apache_avro::types::{Value, ValueKind}; use itertools::Itertools; use risingwave_common::bail; use risingwave_common::log::LogSuppresser; @@ -22,13 +23,20 @@ use risingwave_common::types::{DataType, Decimal}; use risingwave_pb::plan_common::{AdditionalColumn, ColumnDesc, ColumnDescVersion}; use crate::error::ConnectorResult; +use crate::parser::unified::bail_uncategorized; +use crate::parser::{AccessError, MapHandling}; -pub fn avro_schema_to_column_descs(schema: &Schema) -> ConnectorResult> { +pub fn avro_schema_to_column_descs( + schema: &Schema, + map_handling: Option, +) -> ConnectorResult> { if let Schema::Record(RecordSchema { fields, .. }) = schema { let mut index = 0; let fields = fields .iter() - .map(|field| avro_field_to_column_desc(&field.name, &field.schema, &mut index)) + .map(|field| { + avro_field_to_column_desc(&field.name, &field.schema, &mut index, map_handling) + }) .collect::>>()?; Ok(fields) } else { @@ -43,8 +51,9 @@ fn avro_field_to_column_desc( name: &str, schema: &Schema, index: &mut i32, + map_handling: Option, ) -> ConnectorResult { - let data_type = avro_type_mapping(schema)?; + let data_type = avro_type_mapping(schema, map_handling)?; match schema { Schema::Record(RecordSchema { name: schema_name, @@ -53,7 +62,7 @@ fn avro_field_to_column_desc( }) => { let vec_column = fields .iter() - .map(|f| avro_field_to_column_desc(&f.name, &f.schema, index)) + .map(|f| avro_field_to_column_desc(&f.name, &f.schema, index, map_handling)) .collect::>>()?; *index += 1; Ok(ColumnDesc { @@ -83,7 +92,10 @@ fn avro_field_to_column_desc( } } -fn avro_type_mapping(schema: &Schema) -> ConnectorResult { +fn avro_type_mapping( + schema: &Schema, + map_handling: Option, +) -> ConnectorResult { let data_type = match schema { Schema::String => DataType::Varchar, Schema::Int => DataType::Int32, @@ -125,13 +137,13 @@ fn avro_type_mapping(schema: &Schema) -> ConnectorResult { let struct_fields = fields .iter() - .map(|f| avro_type_mapping(&f.schema)) + .map(|f| avro_type_mapping(&f.schema, map_handling)) .collect::>>()?; let struct_names = fields.iter().map(|f| f.name.clone()).collect_vec(); DataType::new_struct(struct_fields, struct_names) } Schema::Array(item_schema) => { - let item_type = avro_type_mapping(item_schema.as_ref())?; + let item_type = avro_type_mapping(item_schema.as_ref(), map_handling)?; DataType::List(Box::new(item_type)) } Schema::Union(union_schema) => { @@ -141,7 +153,7 @@ fn avro_type_mapping(schema: &Schema) -> ConnectorResult { .find_or_first(|s| !matches!(s, Schema::Null)) .ok_or_else(|| anyhow::format_err!("unsupported Avro type: {:?}", union_schema))?; - avro_type_mapping(nested_schema)? + avro_type_mapping(nested_schema, map_handling)? } Schema::Ref { name } => { if name.name == DBZ_VARIABLE_SCALE_DECIMAL_NAME @@ -152,10 +164,153 @@ fn avro_type_mapping(schema: &Schema) -> ConnectorResult { bail!("unsupported Avro type: {:?}", schema); } } - Schema::Map(_) | Schema::Null | Schema::Fixed(_) | Schema::Uuid => { + Schema::Map(value_schema) => { + // TODO: support native map type + match map_handling { + Some(MapHandling::Jsonb) => { + if supported_avro_to_json_type(value_schema) { + DataType::Jsonb + } else { + bail!( + "unsupported Avro type, cannot convert map to jsonb: {:?}", + schema + ) + } + } + None => { + bail!("`map.handling.mode` not specified in ENCODE AVRO (...). Currently supported modes: `jsonb`") + } + } + } + Schema::Null | Schema::Fixed(_) | Schema::Uuid => { bail!("unsupported Avro type: {:?}", schema) } }; Ok(data_type) } + +/// Check for [`avro_to_jsonb`] +fn supported_avro_to_json_type(schema: &Schema) -> bool { + match schema { + Schema::Null | Schema::Boolean | Schema::Int | Schema::String => true, + + Schema::Map(value_schema) | Schema::Array(value_schema) => { + supported_avro_to_json_type(value_schema) + } + Schema::Record(RecordSchema { fields, .. }) => fields + .iter() + .all(|f| supported_avro_to_json_type(&f.schema)), + Schema::Long + | Schema::Float + | Schema::Double + | Schema::Bytes + | Schema::Enum(_) + | Schema::Fixed(_) + | Schema::Decimal(_) + | Schema::Uuid + | Schema::Date + | Schema::TimeMillis + | Schema::TimeMicros + | Schema::TimestampMillis + | Schema::TimestampMicros + | Schema::LocalTimestampMillis + | Schema::LocalTimestampMicros + | Schema::Duration + | Schema::Ref { name: _ } + | Schema::Union(_) => false, + } +} + +pub(crate) fn avro_to_jsonb( + avro: &Value, + builder: &mut jsonbb::Builder, +) -> crate::parser::AccessResult<()> { + match avro { + Value::Null => builder.add_null(), + Value::Boolean(b) => builder.add_bool(*b), + Value::Int(i) => builder.add_i64(*i as i64), + Value::String(s) => builder.add_string(s), + Value::Map(m) => { + builder.begin_object(); + for (k, v) in m { + builder.add_string(k); + avro_to_jsonb(v, builder)?; + } + builder.end_object() + } + // same representation as map + Value::Record(r) => { + builder.begin_object(); + for (k, v) in r { + builder.add_string(k); + avro_to_jsonb(v, builder)?; + } + builder.end_object() + } + Value::Array(a) => { + builder.begin_array(); + for v in a { + avro_to_jsonb(v, builder)?; + } + builder.end_array() + } + + // TODO: figure out where the following encoding is reasonable before enabling them. + // See discussions: https://github.com/risingwavelabs/risingwave/pull/16948 + + // jsonbb supports int64, but JSON spec does not allow it. How should we handle it? + // BTW, protobuf canonical JSON converts int64 to string. + // Value::Long(l) => builder.add_i64(*l), + // Value::Float(f) => { + // if f.is_nan() || f.is_infinite() { + // // XXX: pad null or return err here? + // builder.add_null() + // } else { + // builder.add_f64(*f as f64) + // } + // } + // Value::Double(f) => { + // if f.is_nan() || f.is_infinite() { + // // XXX: pad null or return err here? + // builder.add_null() + // } else { + // builder.add_f64(*f) + // } + // } + // // XXX: What encoding to use? + // // ToText is \x plus hex string. + // Value::Bytes(b) => builder.add_string(&ToText::to_text(&b.as_slice())), + // Value::Enum(_, symbol) => { + // builder.add_string(&symbol); + // } + // Value::Uuid(id) => builder.add_string(&id.as_hyphenated().to_string()), + // // For Union, one concern is that the avro union is tagged (like rust enum) but json union is untagged (like c union). + // // When the union consists of multiple records, it is possible to distinguish which variant is active in avro, but in json they will all become jsonb objects and indistinguishable. + // Value::Union(_, v) => avro_to_jsonb(v, builder)? + // XXX: pad null or return err here? + v @ (Value::Long(_) + | Value::Float(_) + | Value::Double(_) + | Value::Bytes(_) + | Value::Enum(_, _) + | Value::Fixed(_, _) + | Value::Date(_) + | Value::Decimal(_) + | Value::TimeMillis(_) + | Value::TimeMicros(_) + | Value::TimestampMillis(_) + | Value::TimestampMicros(_) + | Value::LocalTimestampMillis(_) + | Value::LocalTimestampMicros(_) + | Value::Duration(_) + | Value::Uuid(_) + | Value::Union(_, _)) => { + bail_uncategorized!( + "unimplemented conversion from avro to jsonb: {:?}", + ValueKind::from(v) + ) + } + } + Ok(()) +} diff --git a/src/connector/src/parser/debezium/avro_parser.rs b/src/connector/src/parser/debezium/avro_parser.rs index 3b11a1926f107..8d73a789b2669 100644 --- a/src/connector/src/parser/debezium/avro_parser.rs +++ b/src/connector/src/parser/debezium/avro_parser.rs @@ -123,14 +123,22 @@ impl DebeziumAvroParserConfig { } pub fn extract_pks(&self) -> ConnectorResult> { - avro_schema_to_column_descs(&self.key_schema) + avro_schema_to_column_descs( + &self.key_schema, + // TODO: do we need to support map type here? + None, + ) } pub fn map_to_columns(&self) -> ConnectorResult> { - avro_schema_to_column_descs(avro_schema_skip_union(avro_extract_field_schema( - &self.outer_schema, - Some("before"), - )?)?) + avro_schema_to_column_descs( + avro_schema_skip_union(avro_extract_field_schema( + &self.outer_schema, + Some("before"), + )?)?, + // TODO: do we need to support map type here? + None, + ) } } @@ -242,7 +250,7 @@ mod tests { } "#; let key_schema = Schema::parse_str(key_schema_str).unwrap(); - let names: Vec = avro_schema_to_column_descs(&key_schema) + let names: Vec = avro_schema_to_column_descs(&key_schema, None) .unwrap() .drain(..) .map(|d| d.name) @@ -298,7 +306,7 @@ mod tests { } "#; let schema = Schema::parse_str(test_schema_str).unwrap(); - let columns = avro_schema_to_column_descs(&schema).unwrap(); + let columns = avro_schema_to_column_descs(&schema, None).unwrap(); for col in &columns { let dtype = col.column_type.as_ref().unwrap(); println!("name = {}, type = {:?}", col.name, dtype.type_name); @@ -316,6 +324,7 @@ mod tests { avro_extract_field_schema(&outer_schema, Some("before")).unwrap(), ) .unwrap(), + None, ) .unwrap() .into_iter() diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index dff681687bc47..c02ae8f655150 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -530,7 +530,7 @@ mod tests { // For other overflow, the parsing succeeds but the type conversion fails // The errors are ignored and logged. res.unwrap(); - assert!(logs_contain("Expected type"), "{i}"); + assert!(logs_contain("expected type"), "{i}"); } else { // For f64 overflow, the parsing fails let e = res.unwrap_err(); diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index f010b8e6b7df6..c7208d272581d 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -174,7 +174,8 @@ pub async fn schema_to_columns( let context = Context::default(); let avro_schema = convert_avro(&json_schema, context).to_string(); let schema = Schema::parse_str(&avro_schema).context("failed to parse avro schema")?; - avro_schema_to_column_descs(&schema) + // TODO: do we need to support map type here? + avro_schema_to_column_descs(&schema, None) } impl ByteStreamSourceParser for JsonParser { diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index eacae886cb076..a249807c6a6bb 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -57,6 +57,7 @@ use crate::parser::util::{ extract_header_inner_from_meta, extract_headers_from_meta, extreact_timestamp_from_meta, }; use crate::schema::schema_registry::SchemaRegistryAuth; +use crate::schema::InvalidOptionError; use crate::source::monitor::GLOBAL_SOURCE_METRICS; use crate::source::{ extract_source_struct, BoxSourceStream, ChunkSourceStream, SourceColumnDesc, SourceColumnType, @@ -985,6 +986,36 @@ pub struct AvroProperties { pub record_name: Option, pub key_record_name: Option, pub name_strategy: PbSchemaRegistryNameStrategy, + pub map_handling: Option, +} + +/// How to convert the map type from the input encoding to RisingWave's datatype. +/// +/// XXX: Should this be `avro.map.handling.mode`? Can it be shared between Avro and Protobuf? +#[derive(Debug, Copy, Clone)] +pub enum MapHandling { + Jsonb, + // TODO: + // Map +} + +impl MapHandling { + pub const OPTION_KEY: &'static str = "map.handling.mode"; + + pub fn from_options( + options: &std::collections::BTreeMap, + ) -> Result, InvalidOptionError> { + let mode = match options.get(Self::OPTION_KEY).map(std::ops::Deref::deref) { + Some("jsonb") => Self::Jsonb, + Some(v) => { + return Err(InvalidOptionError { + message: format!("unrecognized {} value {}", Self::OPTION_KEY, v), + }) + } + None => return Ok(None), + }; + Ok(Some(mode)) + } } #[derive(Debug, Default, Clone)] @@ -1091,6 +1122,7 @@ impl SpecificParserConfig { .unwrap(), use_schema_registry: info.use_schema_registry, row_schema_location: info.row_schema_location.clone(), + map_handling: MapHandling::from_options(&info.format_encode_options)?, ..Default::default() }; if format == SourceFormat::Upsert { diff --git a/src/connector/src/parser/unified/avro.rs b/src/connector/src/parser/unified/avro.rs index 837b879f92d29..bbab918f5be1d 100644 --- a/src/connector/src/parser/unified/avro.rs +++ b/src/connector/src/parser/unified/avro.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::str::FromStr; use std::sync::LazyLock; use apache_avro::schema::{DecimalSchema, RecordSchema}; @@ -31,6 +30,7 @@ use risingwave_common::util::iter_util::ZipEqFast; use super::{bail_uncategorized, uncategorized, Access, AccessError, AccessResult}; use crate::error::ConnectorResult; +use crate::parser::avro::util::avro_to_jsonb; #[derive(Clone)] /// Options for parsing an `AvroValue` into Datum, with an optional avro schema. pub struct AvroParseOptions<'a> { @@ -266,8 +266,12 @@ impl<'a> AvroParseOptions<'a> { value.clone().into_boxed_slice().into() } // ---- Jsonb ----- - (Some(DataType::Jsonb), Value::String(s)) => { - JsonbVal::from_str(s).map_err(|_| create_error())?.into() + (Some(DataType::Jsonb), v @ Value::Map(_)) => { + let mut builder = jsonbb::Builder::default(); + avro_to_jsonb(v, &mut builder)?; + let jsonb = builder.finish(); + debug_assert!(jsonb.as_ref().is_object()); + JsonbVal::from(jsonb).into() } (_expected, _got) => Err(create_error())?, @@ -308,12 +312,6 @@ where options.schema = options.extract_inner_schema(None); continue; } - Value::Map(fields) if fields.contains_key(key) => { - value = fields.get(key).unwrap(); - options.schema = None; - i += 1; - continue; - } Value::Record(fields) => { if let Some((_, v)) = fields.iter().find(|(k, _)| k == key) { value = v; @@ -420,7 +418,8 @@ pub fn avro_extract_field_schema<'a>( } Schema::Array(schema) => Ok(schema), Schema::Union(_) => avro_schema_skip_union(schema), - _ => bail!("avro schema is not a record or array"), + Schema::Map(schema) => Ok(schema), + _ => bail!("avro schema does not have inner item, schema: {:?}", schema), } } @@ -430,6 +429,8 @@ pub(crate) fn unix_epoch_days() -> i32 { #[cfg(test)] mod tests { + use std::str::FromStr; + use apache_avro::Decimal as AvroDecimal; use risingwave_common::types::Decimal; diff --git a/src/connector/src/parser/unified/mod.rs b/src/connector/src/parser/unified/mod.rs index f54d48e4e0b98..b206ddd76f0b2 100644 --- a/src/connector/src/parser/unified/mod.rs +++ b/src/connector/src/parser/unified/mod.rs @@ -95,7 +95,7 @@ where pub enum AccessError { #[error("Undefined field `{name}` at `{path}`")] Undefined { name: String, path: String }, - #[error("Expected type `{expected}` but got `{got}` for `{value}`")] + #[error("Cannot parse value `{value}` with type `{got}` into expected type `{expected}`")] TypeError { expected: String, got: String, diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index ffacf05faf90d..bd77ea68ce8a8 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -446,6 +446,7 @@ pub(crate) async fn bind_columns_from_source( // Parse the value but throw it away. // It would be too late to report error in `SpecificParserConfig::new`, // which leads to recovery loop. + // TODO: rely on SpecificParserConfig::new to validate, like Avro TimestamptzHandling::from_options(&format_encode_options_to_consume) .map_err(|err| InvalidInputSyntax(err.message))?; try_consume_string_from_options( From 44306fd337f550f14ed8e9d7d0c98a5bb16476f0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 28 May 2024 16:17:23 +0800 Subject: [PATCH 06/13] chore(deps): Bump the aws group across 1 directory with 4 updates (#16913) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 1f6bacca0fc81..ab2f45a519bae 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1186,9 +1186,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.1.3" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d67c6836a1009b23e3f4cd1457c83e0aa49a490d9c3033b53c3f7b8cf2facc0f" +checksum = "e16838e6c9e12125face1c1eff1343c75e3ff540de98ff7ebd61874a89bcfeb9" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -1383,9 +1383,9 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "1.1.3" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2eac0bb78e9e2765699999a02d7bfb4e6ad8f13e0962ebb9f5202b1d8cd76006" +checksum = "62220bc6e97f946ddd51b5f1361f78996e704677afc518a4ff66b7a72ea1378c" dependencies = [ "futures-util", "pin-project-lite", @@ -1491,24 +1491,26 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.1.3" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02ca2da7619517310bfead6d18abcdde90f1439224d887d608503cfacff46dff" +checksum = "5b7d790d553d163c7d80a4e06e2906bf24b9172c9ebe045fc3a274e9358ab7bb" dependencies = [ "aws-smithy-async", "aws-smithy-types", "bytes", "http 0.2.9", + "http 1.0.0", "pin-project-lite", "tokio", "tracing", + "zeroize", ] [[package]] name = "aws-smithy-types" -version = "1.1.3" +version = "1.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d4bb944488536cd2fef43212d829bc7e9a8bfc4afa079d21170441e7be8d2d0" +checksum = "5b6764ba7e1c5ede1c9f9e4046645534f06c2581402461c559b481a420330a83" dependencies = [ "base64-simd 0.8.0", "bytes", @@ -1549,9 +1551,9 @@ dependencies = [ [[package]] name = "aws-types" -version = "1.1.3" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ee2739d97d47f47cdf0d27982019a405dcc736df25925d1a75049f1faa79df88" +checksum = "02fa328e19c849b20ef7ada4c9b581dd12351ff35ecc7642d06e69de4f98407c" dependencies = [ "aws-credential-types", "aws-smithy-async", @@ -9372,7 +9374,7 @@ dependencies = [ "indoc", "libc", "memoffset", - "parking_lot 0.12.1", + "parking_lot 0.11.2", "portable-atomic", "pyo3-build-config", "pyo3-ffi", From e2bdd4feb1ce336013c97fc06629a807f33b1b3a Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 28 May 2024 16:38:49 +0800 Subject: [PATCH 07/13] feat(streaming): lower high join amplification logging to > 2048 records, and rate limit at 1 per minute (#16957) --- src/common/src/config.rs | 9 +++++++++ src/config/example.toml | 1 + src/stream/src/executor/hash_join.rs | 24 +++++++++++++++++++----- src/stream/src/from_proto/hash_join.rs | 7 +++++++ 4 files changed, 36 insertions(+), 5 deletions(-) diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 99aae245125eb..cd276e8a966f4 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -976,6 +976,11 @@ pub struct StreamingDeveloperConfig { /// If true, the arrangement backfill will be disabled, /// even if session variable set. pub enable_arrangement_backfill: bool, + + #[serde(default = "default::developer::stream_high_join_amplification_threshold")] + /// If number of hash join matches exceeds this threshold number, + /// it will be logged. + pub high_join_amplification_threshold: usize, } /// The subsections `[batch.developer]`. @@ -1733,6 +1738,10 @@ pub mod default { pub fn stream_enable_arrangement_backfill() -> bool { true } + + pub fn stream_high_join_amplification_threshold() -> usize { + 2048 + } } pub use crate::system_param::default as system; diff --git a/src/config/example.toml b/src/config/example.toml index fc70258788bbc..a50b4b8c10d65 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -118,6 +118,7 @@ stream_memory_controller_eviction_factor_stable = 1.0 stream_memory_controller_sequence_tls_step = 128 stream_memory_controller_sequence_tls_lag = 32 stream_enable_arrangement_backfill = true +stream_high_join_amplification_threshold = 2048 [storage] share_buffers_sync_parallelism = 1 diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index 264f6cbb67a81..0bf825138d92a 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -11,11 +11,12 @@ // 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, HashSet}; +use std::num::NonZeroU32; use std::sync::LazyLock; use std::time::Duration; +use governor::{Quota, RateLimiter}; use itertools::Itertools; use multimap::MultiMap; use risingwave_common::array::Op; @@ -159,6 +160,8 @@ pub struct HashJoinExecutor `BufferedWatermarks` watermark_buffers: BTreeMap>, + + high_join_amplification_threshold: usize, } impl std::fmt::Debug @@ -195,6 +198,7 @@ struct EqJoinArgs<'a, K: HashKey, S: StateStore> { append_only_optimize: bool, chunk_size: usize, cnt_rows_received: &'a mut u32, + high_join_amplification_threshold: usize, } impl HashJoinExecutor { @@ -218,6 +222,7 @@ impl HashJoinExecutor, chunk_size: usize, + high_join_amplification_threshold: usize, ) -> Self { let side_l_column_n = input_l.schema().len(); @@ -446,6 +451,7 @@ impl HashJoinExecutor HashJoinExecutor HashJoinExecutor HashJoinExecutor HashJoinExecutor= 10000 { - static LOG_SUPPERSSER: LazyLock = - LazyLock::new(LogSuppresser::default); + if rows.len() > high_join_amplification_threshold { + static LOG_SUPPERSSER: LazyLock = LazyLock::new(|| { + LogSuppresser::new(RateLimiter::direct(Quota::per_minute( + NonZeroU32::new(1).unwrap(), + ))) + }); if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { let join_key_data_types = side_update.ht.join_key_data_types(); let key = key.deserialize(join_key_data_types)?; - tracing::warn!(target: "hash_join_amplification", + tracing::warn!(target: "high_join_amplification", suppressed_count, matched_rows_len = rows.len(), update_table_id = side_update.ht.table_id(), @@ -1205,6 +1217,7 @@ mod tests { false, Arc::new(StreamingMetrics::unused()), 1024, + 2048, ); (tx_l, tx_r, executor.boxed().execute()) } @@ -1297,6 +1310,7 @@ mod tests { true, Arc::new(StreamingMetrics::unused()), 1024, + 2048, ); (tx_l, tx_r, executor.boxed().execute()) } diff --git a/src/stream/src/from_proto/hash_join.rs b/src/stream/src/from_proto/hash_join.rs index 28a6aa72d4a6f..2d421274cec39 100644 --- a/src/stream/src/from_proto/hash_join.rs +++ b/src/stream/src/from_proto/hash_join.rs @@ -155,6 +155,11 @@ impl ExecutorBuilder for HashJoinExecutorBuilder { join_type_proto: node.get_join_type()?, join_key_data_types, chunk_size: params.env.config().developer.chunk_size, + high_join_amplification_threshold: params + .env + .config() + .developer + .high_join_amplification_threshold, }; let exec = args.dispatch()?; @@ -183,6 +188,7 @@ struct HashJoinExecutorDispatcherArgs { join_type_proto: JoinTypeProto, join_key_data_types: Vec, chunk_size: usize, + high_join_amplification_threshold: usize, } impl HashKeyDispatcher for HashJoinExecutorDispatcherArgs { @@ -211,6 +217,7 @@ impl HashKeyDispatcher for HashJoinExecutorDispatcherArgs { self.is_append_only, self.metrics, self.chunk_size, + self.high_join_amplification_threshold, ) .boxed()) }; From 84b0ac2f750c99fa167a482ecd531963cd8e6407 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 28 May 2024 21:58:17 +0800 Subject: [PATCH 08/13] feat(ci): introduce slow e2e test step (#16953) --- ci/scripts/run-e2e-test.sh | 2 - ci/scripts/slow-e2e-test.sh | 40 +++++++++++++++++++ ci/workflows/main-cron.yml | 19 +++++++++ ci/workflows/pull-request.yml | 19 +++++++++ .../backfill/rate_limit/slow-udf.slt | 0 .../udf/always_retry_python.slt | 6 +-- 6 files changed, 81 insertions(+), 5 deletions(-) create mode 100755 ci/scripts/slow-e2e-test.sh rename {e2e_test => e2e_test_slow}/backfill/rate_limit/slow-udf.slt (100%) rename {e2e_test => e2e_test_slow}/udf/always_retry_python.slt (93%) diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index ef11428ad5d00..793b12dde8b2c 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -80,7 +80,6 @@ RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=i cluster_start # Please make sure the regression is expected before increasing the timeout. sqllogictest -p 4566 -d dev './e2e_test/streaming/**/*.slt' --junit "streaming-${profile}" -sqllogictest -p 4566 -d dev './e2e_test/backfill/rate_limit/*.slt' sqllogictest -p 4566 -d dev './e2e_test/backfill/sink/different_pk_and_dist_key.slt' echo "--- Kill cluster" @@ -115,7 +114,6 @@ pkill python3 sqllogictest -p 4566 -d dev './e2e_test/udf/alter_function.slt' sqllogictest -p 4566 -d dev './e2e_test/udf/graceful_shutdown_python.slt' -sqllogictest -p 4566 -d dev './e2e_test/udf/always_retry_python.slt' # FIXME: flaky test # sqllogictest -p 4566 -d dev './e2e_test/udf/retry_python.slt' diff --git a/ci/scripts/slow-e2e-test.sh b/ci/scripts/slow-e2e-test.sh new file mode 100755 index 0000000000000..2fa1c977a9cfa --- /dev/null +++ b/ci/scripts/slow-e2e-test.sh @@ -0,0 +1,40 @@ +#!/usr/bin/env bash + +# Exits as soon as any line fails. +set -euo pipefail + +while getopts 'p:m:' opt; do + case ${opt} in + p ) + profile=$OPTARG + ;; + m ) + mode=$OPTARG + ;; + \? ) + echo "Invalid Option: -$OPTARG" 1>&2 + exit 1 + ;; + : ) + echo "Invalid option: $OPTARG requires an argument" 1>&2 + ;; + esac +done +shift $((OPTIND -1)) + +source ci/scripts/common.sh + +download_and_prepare_rw "$profile" common + + +echo "--- Download artifacts" +mkdir -p e2e_test/udf/wasm/target/wasm32-wasi/release/ +buildkite-agent artifact download udf.wasm e2e_test/udf/wasm/target/wasm32-wasi/release/ +buildkite-agent artifact download udf.jar ./ + +echo "--- e2e, $mode, slow-udf-tests" +python3 -m pip install --break-system-packages arrow-udf==0.2.1 +RUST_LOG="info" \ +risedev ci-start "$mode" +sqllogictest -p 4566 -d dev './e2e_test_slow/udf/always_retry_python.slt' +sqllogictest -p 4566 -d dev './e2e_test_slow/backfill/rate_limit/slow-udf.slt' diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 1cbdefb6e7540..d9ddb954b2f17 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -90,6 +90,25 @@ steps: timeout_in_minutes: 15 retry: *auto-retry + - label: "slow end-to-end test (release)" + key: "slow-e2e-test-release" + command: "ci/scripts/slow-e2e-test.sh -p ci-release -m ci-3streaming-2serving-3fe" + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-slow-e2e-tests" + || build.env("CI_STEPS") =~ /(^|,)slow-e2e-tests?(,|$$)/ + depends_on: + - "build" + - "build-other" + plugins: + - docker-compose#v5.1.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 4 + retry: *auto-retry + - label: "meta backup test (release)" key: "e2e-meta-backup-test-release" command: "ci/scripts/run-meta-backup-test.sh -p ci-release -m ci-3streaming-2serving-3fe" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 7da2b502fbb0e..2f91e23289da0 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -118,6 +118,25 @@ steps: timeout_in_minutes: 23 retry: *auto-retry + - label: "slow end-to-end test" + key: "slow-e2e-test" + command: "ci/scripts/slow-e2e-test.sh -p ci-dev -m ci-3streaming-2serving-3fe" + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-slow-e2e-tests" + || build.env("CI_STEPS") =~ /(^|,)slow-e2e-tests?(,|$$)/ + depends_on: + - "build" + - "build-other" + plugins: + - docker-compose#v5.1.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 6 + retry: *auto-retry + - label: "end-to-end test (parallel)" command: "ci/scripts/e2e-test-parallel.sh -p ci-dev" if: | diff --git a/e2e_test/backfill/rate_limit/slow-udf.slt b/e2e_test_slow/backfill/rate_limit/slow-udf.slt similarity index 100% rename from e2e_test/backfill/rate_limit/slow-udf.slt rename to e2e_test_slow/backfill/rate_limit/slow-udf.slt diff --git a/e2e_test/udf/always_retry_python.slt b/e2e_test_slow/udf/always_retry_python.slt similarity index 93% rename from e2e_test/udf/always_retry_python.slt rename to e2e_test_slow/udf/always_retry_python.slt index 31179c13f7d45..1889bd136bb97 100644 --- a/e2e_test/udf/always_retry_python.slt +++ b/e2e_test_slow/udf/always_retry_python.slt @@ -2,7 +2,7 @@ system ok python3 e2e_test/udf/test.py & # wait for server to start -sleep 5s +sleep 10s statement ok CREATE FUNCTION sleep_always_retry(INT) RETURNS INT AS 'sleep' USING LINK 'http://localhost:8815' WITH ( always_retry_on_network_error = true ); @@ -33,9 +33,9 @@ CREATE MATERIALIZED VIEW mv_no_retry AS SELECT sleep_no_retry(v1) as s1 from t; statement ok CREATE MATERIALIZED VIEW mv_always_retry AS SELECT sleep_always_retry(v1) as s1 from t; -# Immediately kill the server, sleep for 10s. +# Immediately kill the server, sleep system ok -pkill -9 -i python && sleep 10 +pkill -9 -i python && sleep 60 # Restart the server system ok From 716264c20792ce3ca7eb8d48e18c2b266e30d9ee Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 29 May 2024 00:27:29 +0800 Subject: [PATCH 09/13] refactor(source): remove some useless indirection (#16970) Signed-off-by: xxchan --- src/connector/src/parser/json_parser.rs | 5 ++- src/connector/src/parser/plain_parser.rs | 16 +++---- src/connector/src/parser/unified/upsert.rs | 51 ++++++++++------------ src/connector/src/parser/unified/util.rs | 9 +--- 4 files changed, 33 insertions(+), 48 deletions(-) diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index c7208d272581d..3621fbc2724b3 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -22,13 +22,13 @@ use risingwave_common::{bail, try_match_expand}; use risingwave_pb::plan_common::ColumnDesc; use super::avro::schema_resolver::ConfluentSchemaResolver; +use super::unified::Access; use super::util::{bytes_from_url, get_kafka_topic}; use super::{EncodingProperties, JsonProperties, SchemaRegistryAuth, SpecificParserConfig}; use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::avro::util::avro_schema_to_column_descs; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; -use crate::parser::unified::util::apply_row_accessor_on_stream_chunk_writer; use crate::parser::unified::AccessImpl; use crate::parser::{ AccessBuilder, ByteStreamSourceParser, ParserFormat, SourceStreamChunkRowWriter, @@ -132,7 +132,8 @@ impl JsonParser { let mut errors = Vec::new(); for value in values { let accessor = JsonAccess::new(value); - match apply_row_accessor_on_stream_chunk_writer(accessor, &mut writer) { + match writer.insert(|column| accessor.access(&[&column.name], Some(&column.data_type))) + { Ok(_) => {} Err(err) => errors.push(err), } diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index db6fe44874d0b..2241f786cfdd5 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -15,6 +15,7 @@ use risingwave_common::bail; use super::unified::json::TimestamptzHandling; +use super::unified::ChangeEvent; use super::{ AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, SourceStreamChunkRowWriter, SpecificParserConfig, @@ -24,8 +25,7 @@ use crate::parser::bytes_parser::BytesAccessBuilder; use crate::parser::simd_json_parser::DebeziumJsonAccessBuilder; use crate::parser::unified::debezium::parse_transaction_meta; use crate::parser::unified::upsert::UpsertChangeEvent; -use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer_with_op; -use crate::parser::unified::{AccessImpl, ChangeEventOperation}; +use crate::parser::unified::AccessImpl; use crate::parser::upsert_parser::get_key_column_name; use crate::parser::{BytesProperties, ParseResult, ParserFormat}; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMeta}; @@ -106,7 +106,6 @@ impl PlainParser { // reuse upsert component but always insert let mut row_op: UpsertChangeEvent, AccessImpl<'_, '_>> = UpsertChangeEvent::default(); - let change_event_op = ChangeEventOperation::Upsert; if let Some(data) = key && let Some(key_builder) = self.key_builder.as_mut() @@ -119,14 +118,9 @@ impl PlainParser { row_op = row_op.with_value(self.payload_builder.generate_accessor(data).await?); } - Ok( - apply_row_operation_on_stream_chunk_writer_with_op( - row_op, - &mut writer, - change_event_op, - ) - .map(|_| ParseResult::Rows)?, - ) + writer.insert(|column: &SourceColumnDesc| row_op.access_field(column))?; + + Ok(ParseResult::Rows) } } diff --git a/src/connector/src/parser/unified/upsert.rs b/src/connector/src/parser/unified/upsert.rs index 9129f0d16d864..dbef878fc2685 100644 --- a/src/connector/src/parser/unified/upsert.rs +++ b/src/connector/src/parser/unified/upsert.rs @@ -60,33 +60,30 @@ impl UpsertChangeEvent { } } -impl Access for UpsertChangeEvent +impl UpsertChangeEvent where K: Access, V: Access, { - fn access(&self, path: &[&str], type_expected: Option<&DataType>) -> super::AccessResult { - let create_error = |name: String| AccessError::Undefined { - name, - path: String::new(), - }; - match path.first() { - Some(&"key") => { - if let Some(ka) = &self.key_accessor { - ka.access(&path[1..], type_expected) - } else { - Err(create_error("key".to_string())) - } - } - Some(&"value") => { - if let Some(va) = &self.value_accessor { - va.access(&path[1..], type_expected) - } else { - Err(create_error("value".to_string())) - } - } - None => Ok(None), - Some(other) => Err(create_error(other.to_string())), + fn access_key(&self, path: &[&str], type_expected: Option<&DataType>) -> super::AccessResult { + if let Some(ka) = &self.key_accessor { + ka.access(path, type_expected) + } else { + Err(AccessError::Undefined { + name: "key".to_string(), + path: String::new(), + }) + } + } + + fn access_value(&self, path: &[&str], type_expected: Option<&DataType>) -> super::AccessResult { + if let Some(va) = &self.value_accessor { + va.access(path, type_expected) + } else { + Err(AccessError::Undefined { + name: "value".to_string(), + path: String::new(), + }) } } } @@ -97,7 +94,7 @@ where V: Access, { fn op(&self) -> std::result::Result { - if let Ok(Some(_)) = self.access(&["value"], None) { + if let Ok(Some(_)) = self.access_value(&[], None) { Ok(ChangeEventOperation::Upsert) } else { Ok(ChangeEventOperation::Delete) @@ -110,12 +107,12 @@ where if let Some(key_as_column_name) = &self.key_column_name && &desc.name == key_as_column_name { - self.access(&["key"], Some(&desc.data_type)) + self.access_key(&[], Some(&desc.data_type)) } else { - self.access(&["key", &desc.name], Some(&desc.data_type)) + self.access_key(&[&desc.name], Some(&desc.data_type)) } } - None => self.access(&["value", &desc.name], Some(&desc.data_type)), + None => self.access_value(&[&desc.name], Some(&desc.data_type)), _ => unreachable!(), } } diff --git a/src/connector/src/parser/unified/util.rs b/src/connector/src/parser/unified/util.rs index 948190edf685a..7641e5f0b2432 100644 --- a/src/connector/src/parser/unified/util.rs +++ b/src/connector/src/parser/unified/util.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use super::{Access, AccessResult, ChangeEvent}; +use super::{AccessResult, ChangeEvent}; use crate::parser::unified::ChangeEventOperation; use crate::parser::SourceStreamChunkRowWriter; use crate::source::SourceColumnDesc; @@ -36,10 +36,3 @@ pub fn apply_row_operation_on_stream_chunk_writer( let op = row_op.op()?; apply_row_operation_on_stream_chunk_writer_with_op(row_op, writer, op) } - -pub fn apply_row_accessor_on_stream_chunk_writer( - accessor: impl Access, - writer: &mut SourceStreamChunkRowWriter<'_>, -) -> AccessResult<()> { - writer.insert(|column| accessor.access(&[&column.name], Some(&column.data_type))) -} From 66df1d3754e3756cc33aa6d30a40f10c19c77bc8 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Tue, 28 May 2024 12:22:11 -0500 Subject: [PATCH 10/13] refactor(metrics): use LabelGuarded for vec metrics in StreamingMetrics (#16737) Signed-off-by: Richard Chien Co-authored-by: Richard Chien --- src/common/metrics/src/monitor/connection.rs | 4 +- src/stream/src/common/metrics.rs | 4 +- .../src/executor/aggregation/distinct.rs | 93 +- .../executor/backfill/arrangement_backfill.rs | 29 +- .../src/executor/backfill/cdc/cdc_backfill.rs | 21 +- .../executor/backfill/no_shuffle_backfill.rs | 29 +- src/stream/src/executor/exchange/input.rs | 8 +- src/stream/src/executor/hash_agg.rs | 50 +- src/stream/src/executor/join/hash_join.rs | 50 +- src/stream/src/executor/lookup/impl_.rs | 34 +- src/stream/src/executor/merge.rs | 7 +- .../src/executor/monitor/streaming_stats.rs | 811 +++++++++++++----- src/stream/src/executor/mview/materialize.rs | 58 +- .../src/executor/over_window/general.rs | 49 +- src/stream/src/executor/receiver.rs | 7 +- src/stream/src/executor/simple_agg.rs | 2 +- src/stream/src/executor/sink.rs | 39 +- .../src/executor/source/fs_source_executor.rs | 21 +- .../source/source_backfill_executor.rs | 19 +- .../src/executor/source/source_executor.rs | 37 +- src/stream/src/executor/temporal_join.rs | 38 +- src/stream/src/executor/top_n/group_top_n.rs | 56 +- .../executor/top_n/group_top_n_appendonly.rs | 60 +- src/stream/src/executor/utils.rs | 35 - src/stream/src/executor/wrapper/trace.rs | 11 +- src/stream/src/task/stream_manager.rs | 15 +- 26 files changed, 917 insertions(+), 670 deletions(-) diff --git a/src/common/metrics/src/monitor/connection.rs b/src/common/metrics/src/monitor/connection.rs index f5570ff97f60f..295fb6399ba4b 100644 --- a/src/common/metrics/src/monitor/connection.rs +++ b/src/common/metrics/src/monitor/connection.rs @@ -744,7 +744,7 @@ impl MonitorAsyncReadWrite for MonitorAsyncReadWriteImpl { } fn on_read_err(&mut self, err: &Error) { - // No need to store the value returned from with_label_values + // No need to store the value returned from `with_guarded_label_values` // because it is reporting a single error. GLOBAL_CONNECTION_METRICS .io_err_rate @@ -775,7 +775,7 @@ impl MonitorAsyncReadWrite for MonitorAsyncReadWriteImpl { } fn on_write_err(&mut self, err: &Error) { - // No need to store the value returned from with_label_values + // No need to store the value returned from `with_guarded_label_values` // because it is reporting a single error. GLOBAL_CONNECTION_METRICS .io_err_rate diff --git a/src/stream/src/common/metrics.rs b/src/stream/src/common/metrics.rs index 2c82bee7d97d9..fb7267d3fd5d1 100644 --- a/src/stream/src/common/metrics.rs +++ b/src/stream/src/common/metrics.rs @@ -30,13 +30,13 @@ impl MetricsInfo { metrics: Arc, table_id: u32, actor_id: ActorId, - desc: &str, + desc: impl Into, ) -> Self { Self { metrics, table_id: table_id.to_string(), actor_id: actor_id.to_string(), - desc: desc.to_string(), + desc: desc.into(), } } diff --git a/src/stream/src/executor/aggregation/distinct.rs b/src/stream/src/executor/aggregation/distinct.rs index 205f9118e3a76..3433708957aba 100644 --- a/src/stream/src/executor/aggregation/distinct.rs +++ b/src/stream/src/executor/aggregation/distinct.rs @@ -15,34 +15,50 @@ use std::collections::HashMap; use std::marker::PhantomData; use std::sync::atomic::AtomicU64; -use std::sync::Arc; use itertools::Itertools; use risingwave_common::array::{ArrayRef, Op}; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; -use risingwave_common::row::{self, CompactedRow, OwnedRow, Row, RowExt}; -use risingwave_common::types::{ScalarImpl, ScalarRefImpl}; +use risingwave_common::row::{self, CompactedRow, RowExt}; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_storage::StateStore; use super::{AggCall, GroupKey}; use crate::cache::ManagedLruCache; use crate::common::metrics::MetricsInfo; -use crate::common::table::state_table::StateTable; -use crate::executor::{ActorContextRef, StreamExecutorResult}; +use crate::executor::monitor::AggDistinctDedupMetrics; +use crate::executor::prelude::*; type DedupCache = ManagedLruCache>; /// Deduplicater for one distinct column. struct ColumnDeduplicater { cache: DedupCache, + metrics: AggDistinctDedupMetrics, _phantom: PhantomData, } impl ColumnDeduplicater { - fn new(watermark_sequence: Arc, metrics_info: MetricsInfo) -> Self { + fn new( + distinct_col: usize, + dedup_table: &StateTable, + watermark_sequence: Arc, + actor_ctx: &ActorContext, + ) -> Self { + let metrics_info = MetricsInfo::new( + actor_ctx.streaming_metrics.clone(), + dedup_table.table_id(), + actor_ctx.id, + format!("distinct dedup column {}", distinct_col), + ); + let metrics = actor_ctx.streaming_metrics.new_agg_distinct_dedup_metrics( + dedup_table.table_id(), + actor_ctx.id, + actor_ctx.fragment_id, + ); + Self { cache: ManagedLruCache::unbounded(watermark_sequence, metrics_info), + metrics, _phantom: PhantomData, } } @@ -54,7 +70,6 @@ impl ColumnDeduplicater { mut visibilities: Vec<&mut Bitmap>, dedup_table: &mut StateTable, group_key: Option<&GroupKey>, - ctx: ActorContextRef, ) -> StreamExecutorResult<()> { let n_calls = visibilities.len(); @@ -64,9 +79,6 @@ impl ColumnDeduplicater { let mut vis_masks_inv = (0..visibilities.len()) .map(|_| BitmapBuilder::zeroed(column.len())) .collect_vec(); - let actor_id_str = ctx.id.to_string(); - let fragment_id_str = ctx.fragment_id.to_string(); - let table_id_str = dedup_table.table_id().to_string(); for (datum_idx, (op, datum)) in ops.iter().zip_eq_fast(column.iter()).enumerate() { // skip if this item is hidden to all agg calls (this is likely to happen) if !visibilities.iter().any(|vis| vis.is_set(datum_idx)) { @@ -79,19 +91,13 @@ impl ColumnDeduplicater { let cache_key = CompactedRow::from(group_key.map(GroupKey::cache_key).chain(row::once(datum))); - ctx.streaming_metrics - .agg_distinct_total_cache_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) - .inc(); + self.metrics.agg_distinct_total_cache_count.inc(); // TODO(yuhao): avoid this `contains`. // https://github.com/risingwavelabs/risingwave/issues/9233 let mut counts = if self.cache.contains(&cache_key) { self.cache.get_mut(&cache_key).unwrap() } else { - ctx.streaming_metrics - .agg_distinct_cache_miss_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) - .inc(); + self.metrics.agg_distinct_cache_miss_count.inc(); // load from table into the cache let counts = if let Some(counts_row) = dedup_table.get_row(&table_pk).await? as Option @@ -180,18 +186,14 @@ impl ColumnDeduplicater { } /// Flush the deduplication table. - fn flush(&mut self, dedup_table: &StateTable, ctx: ActorContextRef) { + fn flush(&mut self, _dedup_table: &StateTable) { // TODO(rc): now we flush the table in `dedup` method. // WARN: if you want to change to batching the write to table. please remember to change // `self.cache.evict()` too. self.cache.evict(); - let actor_id_str = ctx.id.to_string(); - let fragment_id_str = ctx.fragment_id.to_string(); - let table_id_str = dedup_table.table_id().to_string(); - ctx.streaming_metrics + self.metrics .agg_distinct_cached_entry_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(self.cache.len() as i64); } } @@ -212,7 +214,6 @@ pub struct DistinctDeduplicater { /// Key: distinct column index; /// Value: (agg call indices that distinct on the column, deduplicater for the column). deduplicaters: HashMap, ColumnDeduplicater)>, - ctx: ActorContextRef, } impl DistinctDeduplicater { @@ -220,9 +221,8 @@ impl DistinctDeduplicater { agg_calls: &[AggCall], watermark_epoch: Arc, distinct_dedup_tables: &HashMap>, - ctx: ActorContextRef, + actor_ctx: &ActorContext, ) -> Self { - let actor_id = ctx.id; let deduplicaters: HashMap<_, _> = agg_calls .iter() .enumerate() @@ -230,19 +230,18 @@ impl DistinctDeduplicater { .into_group_map_by(|(_, call)| call.args.val_indices()[0]) .into_iter() .map(|(distinct_col, indices_and_calls)| { - let table_id = distinct_dedup_tables.get(&distinct_col).unwrap().table_id(); - let metrics_info = MetricsInfo::new( - ctx.streaming_metrics.clone(), - table_id, - actor_id, - "distinct dedup", - ); + let dedup_table = distinct_dedup_tables.get(&distinct_col).unwrap(); let call_indices: Box<[_]> = indices_and_calls.into_iter().map(|v| v.0).collect(); - let deduplicater = ColumnDeduplicater::new(watermark_epoch.clone(), metrics_info); + let deduplicater = ColumnDeduplicater::new( + distinct_col, + dedup_table, + watermark_epoch.clone(), + actor_ctx, + ); (distinct_col, (call_indices, deduplicater)) }) .collect(); - Self { deduplicaters, ctx } + Self { deduplicaters } } pub fn dedup_caches_mut(&mut self) -> impl Iterator { @@ -269,14 +268,7 @@ impl DistinctDeduplicater { // SAFETY: all items in `agg_call_indices` are unique by nature, see `new`. let visibilities = unsafe { get_many_mut_from_slice(&mut visibilities, call_indices) }; deduplicater - .dedup( - ops, - column, - visibilities, - dedup_table, - group_key, - self.ctx.clone(), - ) + .dedup(ops, column, visibilities, dedup_table, group_key) .await?; } Ok(visibilities) @@ -289,7 +281,7 @@ impl DistinctDeduplicater { ) -> StreamExecutorResult<()> { for (distinct_col, (_, deduplicater)) in &mut self.deduplicaters { let dedup_table = dedup_tables.get_mut(distinct_col).unwrap(); - deduplicater.flush(dedup_table, self.ctx.clone()); + deduplicater.flush(dedup_table); } Ok(()) } @@ -297,16 +289,13 @@ impl DistinctDeduplicater { #[cfg(test)] mod tests { - use risingwave_common::array::StreamChunk; use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId}; use risingwave_common::test_prelude::StreamChunkTestExt; - use risingwave_common::types::DataType; use risingwave_common::util::epoch::{test_epoch, EpochPair}; use risingwave_common::util::sort_util::OrderType; use risingwave_storage::memory::MemoryStateStore; use super::*; - use crate::executor::ActorContext; async fn infer_dedup_tables( agg_calls: &[AggCall], @@ -392,7 +381,7 @@ mod tests { &agg_calls, Arc::new(AtomicU64::new(0)), &dedup_tables, - ActorContext::for_test(0), + &ActorContext::for_test(0), ); // --- chunk 1 --- @@ -483,7 +472,7 @@ mod tests { &agg_calls, Arc::new(AtomicU64::new(0)), &dedup_tables, - ActorContext::for_test(0), + &ActorContext::for_test(0), ); // --- chunk 3 --- @@ -572,7 +561,7 @@ mod tests { &agg_calls, Arc::new(AtomicU64::new(0)), &dedup_tables, - ActorContext::for_test(0), + &ActorContext::for_test(0), ); let chunk = StreamChunk::from_pretty( diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 9d58f33d430bc..557dda4f535ef 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -207,21 +207,9 @@ where let mut rate_limiter = rate_limit.and_then(create_limiter); - let backfill_snapshot_read_row_count_metric = self + let metrics = self .metrics - .backfill_snapshot_read_row_count - .with_guarded_label_values(&[ - upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), - ]); - - let backfill_upstream_output_row_count_metric = self - .metrics - .backfill_upstream_output_row_count - .with_guarded_label_values(&[ - upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), - ]); + .new_backfill_metrics(upstream_table_id, self.actor_id); 'backfill_loop: loop { let mut cur_barrier_snapshot_processed_rows: u64 = 0; @@ -313,9 +301,11 @@ where &self.output_indices, )); } - backfill_snapshot_read_row_count_metric + metrics + .backfill_snapshot_read_row_count .inc_by(cur_barrier_snapshot_processed_rows); - backfill_upstream_output_row_count_metric + metrics + .backfill_upstream_output_row_count .inc_by(cur_barrier_upstream_processed_rows); break 'backfill_loop; } @@ -458,8 +448,11 @@ where upstream_table.commit(barrier.epoch).await?; - backfill_snapshot_read_row_count_metric.inc_by(cur_barrier_snapshot_processed_rows); - backfill_upstream_output_row_count_metric + metrics + .backfill_snapshot_read_row_count + .inc_by(cur_barrier_snapshot_processed_rows); + metrics + .backfill_upstream_output_row_count .inc_by(cur_barrier_upstream_processed_rows); // Update snapshot read epoch. diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index 584a74a0c03b6..8caffd59b309b 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -40,6 +40,7 @@ use crate::executor::backfill::utils::{ get_cdc_chunk_last_offset, get_new_pos, mapping_chunk, mapping_message, mark_cdc_chunk, }; use crate::executor::backfill::CdcScanOptions; +use crate::executor::monitor::CdcBackfillMetrics; use crate::executor::prelude::*; use crate::task::CreateMviewProgress; @@ -68,7 +69,7 @@ pub struct CdcBackfillExecutor { // This object is just a stub right now progress: Option, - metrics: Arc, + metrics: CdcBackfillMetrics, /// Rate limit in rows/s. rate_limit_rps: Option, @@ -98,6 +99,8 @@ impl CdcBackfillExecutor { pk_indices.len() + METADATA_STATE_LEN, ); + let metrics = metrics.new_cdc_backfill_metrics(external_table.table_id(), actor_ctx.id); + Self { actor_ctx, external_table, @@ -113,26 +116,16 @@ impl CdcBackfillExecutor { } fn report_metrics( - metrics: &Arc, - upstream_table_id: u32, - actor_id: u32, + metrics: &CdcBackfillMetrics, snapshot_processed_row_count: u64, upstream_processed_row_count: u64, ) { metrics .cdc_backfill_snapshot_read_row_count - .with_label_values(&[ - upstream_table_id.to_string().as_str(), - actor_id.to_string().as_str(), - ]) .inc_by(snapshot_processed_row_count); metrics .cdc_backfill_upstream_output_row_count - .with_label_values(&[ - upstream_table_id.to_string().as_str(), - actor_id.to_string().as_str(), - ]) .inc_by(upstream_processed_row_count); } @@ -332,8 +325,6 @@ impl CdcBackfillExecutor { Self::report_metrics( &self.metrics, - upstream_table_id, - self.actor_ctx.id, cur_barrier_snapshot_processed_rows, cur_barrier_upstream_processed_rows, ); @@ -552,8 +543,6 @@ impl CdcBackfillExecutor { Self::report_metrics( &self.metrics, - upstream_table_id, - self.actor_ctx.id, cur_barrier_snapshot_processed_rows, cur_barrier_upstream_processed_rows, ); diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index fdf955fbbdfc7..bd130fd8f52a2 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -209,21 +209,9 @@ where let mut pending_barrier: Option = None; let mut rate_limiter = rate_limit.and_then(create_limiter); - let backfill_snapshot_read_row_count_metric = self + let metrics = self .metrics - .backfill_snapshot_read_row_count - .with_guarded_label_values(&[ - upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), - ]); - - let backfill_upstream_output_row_count_metric = self - .metrics - .backfill_upstream_output_row_count - .with_guarded_label_values(&[ - upstream_table_id.to_string().as_str(), - self.actor_id.to_string().as_str(), - ]); + .new_backfill_metrics(upstream_table_id, self.actor_id); 'backfill_loop: loop { let mut cur_barrier_snapshot_processed_rows: u64 = 0; @@ -311,9 +299,11 @@ where &self.output_indices, )); } - backfill_snapshot_read_row_count_metric + metrics + .backfill_snapshot_read_row_count .inc_by(cur_barrier_snapshot_processed_rows); - backfill_upstream_output_row_count_metric + metrics + .backfill_upstream_output_row_count .inc_by(cur_barrier_upstream_processed_rows); break 'backfill_loop; } @@ -416,8 +406,11 @@ where upstream_chunk_buffer.clear() } - backfill_snapshot_read_row_count_metric.inc_by(cur_barrier_snapshot_processed_rows); - backfill_upstream_output_row_count_metric + metrics + .backfill_snapshot_read_row_count + .inc_by(cur_barrier_snapshot_processed_rows); + metrics + .backfill_upstream_output_row_count .inc_by(cur_barrier_upstream_processed_rows); // Update snapshot read epoch. diff --git a/src/stream/src/executor/exchange/input.rs b/src/stream/src/executor/exchange/input.rs index 0fc4845476db0..4c5648ab1dd49 100644 --- a/src/stream/src/executor/exchange/input.rs +++ b/src/stream/src/executor/exchange/input.rs @@ -155,6 +155,9 @@ impl RemoteInput { let up_actor_id = up_down_ids.0.to_string(); let up_fragment_id = up_down_frag.0.to_string(); let down_fragment_id = up_down_frag.1.to_string(); + let exchange_frag_recv_size_metrics = metrics + .exchange_frag_recv_size + .with_guarded_label_values(&[&up_fragment_id, &down_fragment_id]); let span: await_tree::Span = format!("RemoteInput (actor {up_actor_id})").into(); @@ -167,10 +170,7 @@ impl RemoteInput { let msg = message.unwrap(); let bytes = Message::get_encoded_len(&msg); - metrics - .exchange_frag_recv_size - .with_label_values(&[&up_fragment_id, &down_fragment_id]) - .inc_by(bytes as u64); + exchange_frag_recv_size_metrics.inc_by(bytes as u64); let msg_res = Message::from_protobuf(&msg); if let Some(add_back_permits) = match permits.unwrap().value { diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index 717c42d57925e..2659e2c154c72 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -31,6 +31,7 @@ use super::aggregation::{ agg_call_filter_res, iter_table_storage, AggStateStorage, DistinctDeduplicater, GroupKey, OnlyOutputIfHasInput, }; +use super::monitor::HashAggMetrics; use super::sort_buffer::SortBuffer; use crate::cache::{cache_may_stale, ManagedLruCache}; use crate::common::metrics::MetricsInfo; @@ -133,6 +134,9 @@ impl ExecutorInner { } struct ExecutionVars { + metrics: HashAggMetrics, + + // Stats collected during execution, will be flushed to metrics at the end of each barrier. stats: ExecutionStats, /// Cache for [`AggGroup`]s. `HashKey` -> `AggGroup`. @@ -388,18 +392,11 @@ impl HashAggExecutor { } // Update the metrics. - let actor_id_str = this.actor_ctx.id.to_string(); - let fragment_id_str = this.actor_ctx.fragment_id.to_string(); - let table_id_str = this.intermediate_state_table.table_id().to_string(); - this.actor_ctx - .streaming_metrics + vars.metrics .agg_dirty_groups_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(vars.dirty_groups.len() as i64); - this.actor_ctx - .streaming_metrics + vars.metrics .agg_dirty_groups_heap_size - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(vars.dirty_groups.estimated_heap_size() as i64); Ok(()) @@ -502,34 +499,21 @@ impl HashAggExecutor { vars.agg_group_cache.evict(); } - fn update_metrics(this: &ExecutorInner, vars: &mut ExecutionVars) { - let actor_id_str = this.actor_ctx.id.to_string(); - let fragment_id_str = this.actor_ctx.fragment_id.to_string(); - let table_id_str = this.intermediate_state_table.table_id().to_string(); - this.actor_ctx - .streaming_metrics + fn flush_metrics(_this: &ExecutorInner, vars: &mut ExecutionVars) { + vars.metrics .agg_lookup_miss_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .inc_by(std::mem::take(&mut vars.stats.lookup_miss_count)); - this.actor_ctx - .streaming_metrics + vars.metrics .agg_total_lookup_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .inc_by(std::mem::take(&mut vars.stats.total_lookup_count)); - this.actor_ctx - .streaming_metrics + vars.metrics .agg_cached_entry_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(vars.agg_group_cache.len() as i64); - this.actor_ctx - .streaming_metrics + vars.metrics .agg_chunk_lookup_miss_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .inc_by(std::mem::take(&mut vars.stats.chunk_lookup_miss_count)); - this.actor_ctx - .streaming_metrics + vars.metrics .agg_chunk_total_lookup_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .inc_by(std::mem::take(&mut vars.stats.chunk_total_lookup_count)); } @@ -570,8 +554,14 @@ impl HashAggExecutor { this.actor_ctx.id, "agg intermediate state table", ); + let metrics = this.actor_ctx.streaming_metrics.new_hash_agg_metrics( + this.intermediate_state_table.table_id(), + this.actor_ctx.id, + this.actor_ctx.fragment_id, + ); let mut vars = ExecutionVars { + metrics, stats: ExecutionStats::new(), agg_group_cache: ManagedLruCache::unbounded_with_hasher( this.watermark_sequence.clone(), @@ -583,7 +573,7 @@ impl HashAggExecutor { &this.agg_calls, this.watermark_sequence.clone(), &this.distinct_dedup_tables, - this.actor_ctx.clone(), + &this.actor_ctx, ), buffered_watermarks: vec![None; this.group_key_indices.len()], window_watermark: None, @@ -638,7 +628,7 @@ impl HashAggExecutor { Self::try_flush_data(&mut this).await?; } Message::Barrier(barrier) => { - Self::update_metrics(&this, &mut vars); + Self::flush_metrics(&this, &mut vars); #[for_await] for chunk in Self::flush_data(&mut this, &mut vars) { diff --git a/src/stream/src/executor/join/hash_join.rs b/src/stream/src/executor/join/hash_join.rs index 52f2c94078261..6398a475a6062 100644 --- a/src/stream/src/executor/join/hash_join.rs +++ b/src/stream/src/executor/join/hash_join.rs @@ -105,9 +105,9 @@ pub struct JoinHashMapMetrics { insert_cache_miss_count: usize, // Metrics - join_lookup_total_count_metric: LabelGuardedIntCounter<5>, - join_lookup_miss_count_metric: LabelGuardedIntCounter<5>, - join_insert_cache_miss_count_metrics: LabelGuardedIntCounter<5>, + join_lookup_total_count_metric: LabelGuardedIntCounter<4>, + join_lookup_miss_count_metric: LabelGuardedIntCounter<4>, + join_insert_cache_miss_count_metrics: LabelGuardedIntCounter<4>, } impl JoinHashMapMetrics { @@ -117,37 +117,19 @@ impl JoinHashMapMetrics { fragment_id: FragmentId, side: &'static str, join_table_id: u32, - degree_table_id: u32, ) -> Self { let actor_id = actor_id.to_string(); let fragment_id = fragment_id.to_string(); let join_table_id = join_table_id.to_string(); - let degree_table_id = degree_table_id.to_string(); - let join_lookup_total_count_metric = - metrics.join_lookup_total_count.with_guarded_label_values(&[ - (side), - &join_table_id, - °ree_table_id, - &actor_id, - &fragment_id, - ]); - let join_lookup_miss_count_metric = - metrics.join_lookup_miss_count.with_guarded_label_values(&[ - (side), - &join_table_id, - °ree_table_id, - &actor_id, - &fragment_id, - ]); + let join_lookup_total_count_metric = metrics + .join_lookup_total_count + .with_guarded_label_values(&[(side), &join_table_id, &actor_id, &fragment_id]); + let join_lookup_miss_count_metric = metrics + .join_lookup_miss_count + .with_guarded_label_values(&[(side), &join_table_id, &actor_id, &fragment_id]); let join_insert_cache_miss_count_metrics = metrics .join_insert_cache_miss_count - .with_guarded_label_values(&[ - (side), - &join_table_id, - °ree_table_id, - &actor_id, - &fragment_id, - ]); + .with_guarded_label_values(&[(side), &join_table_id, &actor_id, &fragment_id]); Self { lookup_miss_count: 0, @@ -265,7 +247,6 @@ impl JoinHashMap { ); let join_table_id = state_table.table_id(); - let degree_table_id = degree_table.table_id(); let state = TableInner { pk_indices: state_pk_indices, join_key_indices: state_join_key_indices, @@ -286,7 +267,7 @@ impl JoinHashMap { metrics.clone(), join_table_id, actor_id, - &format!("hash join {}", side), + format!("hash join {}", side), ); let cache = ManagedLruCache::unbounded_with_hasher_in( @@ -305,14 +286,7 @@ impl JoinHashMap { degree_state, need_degree_table, pk_contained_in_jk, - metrics: JoinHashMapMetrics::new( - &metrics, - actor_id, - fragment_id, - side, - join_table_id, - degree_table_id, - ), + metrics: JoinHashMapMetrics::new(&metrics, actor_id, fragment_id, side, join_table_id), } } diff --git a/src/stream/src/executor/lookup/impl_.rs b/src/stream/src/executor/lookup/impl_.rs index 3cdc31eef3cad..2ec47ca3fc500 100644 --- a/src/stream/src/executor/lookup/impl_.rs +++ b/src/stream/src/executor/lookup/impl_.rs @@ -31,6 +31,7 @@ use crate::executor::join::builder::JoinStreamChunkBuilder; use crate::executor::lookup::cache::LookupCache; use crate::executor::lookup::sides::{ArrangeJoinSide, ArrangeMessage, StreamJoinSide}; use crate::executor::lookup::LookupExecutor; +use crate::executor::monitor::LookupExecutorMetrics; use crate::executor::prelude::*; /// Parameters for [`LookupExecutor`]. @@ -238,6 +239,12 @@ impl LookupExecutor { .boxed() }; + let metrics = self.ctx.streaming_metrics.new_lookup_executor_metrics( + self.arrangement.storage_table.table_id(), + self.ctx.id, + self.ctx.fragment_id, + ); + let (stream_to_output, arrange_to_output) = JoinStreamChunkBuilder::get_i2o_mapping( &self.column_mapping, self.stream.col_types.len(), @@ -293,7 +300,11 @@ impl LookupExecutor { for (op, row) in ops.iter().zip_eq_debug(chunk.rows()) { for matched_row in self - .lookup_one_row(&row, self.last_barrier.as_ref().unwrap().epoch) + .lookup_one_row( + &row, + self.last_barrier.as_ref().unwrap().epoch, + &metrics, + ) .await? { tracing::debug!(target: "events::stream::lookup::put", "{:?} {:?}", row, matched_row); @@ -336,30 +347,21 @@ impl LookupExecutor { &mut self, stream_row: &RowRef<'_>, epoch_pair: EpochPair, + metrics: &LookupExecutorMetrics, ) -> StreamExecutorResult> { // stream_row is the row from stream side, we need to transform into the correct order of // the arrangement side. let lookup_row = stream_row .project(&self.key_indices_mapping) .into_owned_row(); - let table_id_str = self.arrangement.storage_table.table_id().to_string(); - let actor_id_str = self.ctx.id.to_string(); - let fragment_id_str = self.ctx.fragment_id.to_string(); - self.ctx - .streaming_metrics - .lookup_total_query_cache_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) - .inc(); + + metrics.lookup_total_query_cache_count.inc(); if let Some(result) = self.lookup_cache.lookup(&lookup_row) { return Ok(result.iter().cloned().collect_vec()); } // cache miss - self.ctx - .streaming_metrics - .lookup_cache_miss_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) - .inc(); + metrics.lookup_cache_miss_count.inc(); tracing::debug!(target: "events::stream::lookup::lookup_row", "{:?}", lookup_row); @@ -404,10 +406,8 @@ impl LookupExecutor { self.lookup_cache.batch_update(lookup_row, all_rows.clone()); - self.ctx - .streaming_metrics + metrics .lookup_cached_entry_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(self.lookup_cache.len() as i64); Ok(all_rows.into_inner()) diff --git a/src/stream/src/executor/merge.rs b/src/stream/src/executor/merge.rs index 49f7cde84ba78..8b1daa9695c4d 100644 --- a/src/stream/src/executor/merge.rs +++ b/src/stream/src/executor/merge.rs @@ -25,7 +25,6 @@ use super::watermark::*; use super::*; use crate::executor::exchange::input::new_input; use crate::executor::prelude::*; -use crate::executor::utils::ActorInputMetrics; use crate::task::SharedContext; /// `MergeExecutor` merges data from multiple channels. Dataflow from one channel @@ -102,8 +101,7 @@ impl MergeExecutor { ); let actor_id = self.actor_context.id; - let mut metrics = ActorInputMetrics::new( - &self.metrics, + let mut metrics = self.metrics.new_actor_input_metrics( actor_id, self.fragment_id, self.upstream_fragment_id, @@ -222,8 +220,7 @@ impl MergeExecutor { } self.upstream_fragment_id = new_upstream_fragment_id; - metrics = ActorInputMetrics::new( - &self.metrics, + metrics = self.metrics.new_actor_input_metrics( actor_id, self.fragment_id, self.upstream_fragment_id, diff --git a/src/stream/src/executor/monitor/streaming_stats.rs b/src/stream/src/executor/monitor/streaming_stats.rs index 6ccde81318ac1..379bdf49a5722 100644 --- a/src/stream/src/executor/monitor/streaming_stats.rs +++ b/src/stream/src/executor/monitor/streaming_stats.rs @@ -14,66 +14,69 @@ use std::sync::OnceLock; -use prometheus::core::{AtomicF64, AtomicI64, AtomicU64, GenericCounterVec, GenericGaugeVec}; use prometheus::{ - exponential_buckets, histogram_opts, register_gauge_vec_with_registry, - register_histogram_with_registry, register_int_counter_vec_with_registry, - register_int_counter_with_registry, register_int_gauge_vec_with_registry, - register_int_gauge_with_registry, Histogram, IntCounter, IntCounterVec, IntGauge, Registry, + exponential_buckets, histogram_opts, register_histogram_with_registry, + register_int_counter_with_registry, register_int_gauge_with_registry, Histogram, IntCounter, + IntGauge, Registry, }; +use risingwave_common::catalog::TableId; use risingwave_common::config::MetricLevel; use risingwave_common::metrics::{ - LabelGuardedHistogramVec, LabelGuardedIntCounterVec, LabelGuardedIntGaugeVec, + LabelGuardedGauge, LabelGuardedGaugeVec, LabelGuardedHistogramVec, LabelGuardedIntCounter, + LabelGuardedIntCounterVec, LabelGuardedIntGauge, LabelGuardedIntGaugeVec, RelabeledGuardedHistogramVec, }; use risingwave_common::monitor::GLOBAL_METRICS_REGISTRY; use risingwave_common::{ - register_guarded_histogram_vec_with_registry, register_guarded_int_counter_vec_with_registry, - register_guarded_int_gauge_vec_with_registry, + register_guarded_gauge_vec_with_registry, register_guarded_histogram_vec_with_registry, + register_guarded_int_counter_vec_with_registry, register_guarded_int_gauge_vec_with_registry, }; +use risingwave_connector::sink::catalog::SinkId; use risingwave_connector::sink::SinkMetrics; use crate::common::log_store_impl::kv_log_store::{ REWIND_BACKOFF_FACTOR, REWIND_BASE_DELAY, REWIND_MAX_DELAY, }; +use crate::executor::prelude::ActorId; +use crate::task::FragmentId; #[derive(Clone)] pub struct StreamingMetrics { pub level: MetricLevel, // Executor metrics (disabled by default) - pub executor_row_count: GenericCounterVec, + pub executor_row_count: LabelGuardedIntCounterVec<3>, // Streaming actor metrics from tokio (disabled by default) - pub actor_execution_time: GenericGaugeVec, - pub actor_scheduled_duration: GenericGaugeVec, - pub actor_scheduled_cnt: GenericGaugeVec, - pub actor_fast_poll_duration: GenericGaugeVec, - pub actor_fast_poll_cnt: GenericGaugeVec, - pub actor_slow_poll_duration: GenericGaugeVec, - pub actor_slow_poll_cnt: GenericGaugeVec, - pub actor_poll_duration: GenericGaugeVec, - pub actor_poll_cnt: GenericGaugeVec, - pub actor_idle_duration: GenericGaugeVec, - pub actor_idle_cnt: GenericGaugeVec, + actor_execution_time: LabelGuardedGaugeVec<1>, + actor_scheduled_duration: LabelGuardedGaugeVec<1>, + actor_scheduled_cnt: LabelGuardedIntGaugeVec<1>, + actor_fast_poll_duration: LabelGuardedGaugeVec<1>, + actor_fast_poll_cnt: LabelGuardedIntGaugeVec<1>, + actor_slow_poll_duration: LabelGuardedGaugeVec<1>, + actor_slow_poll_cnt: LabelGuardedIntGaugeVec<1>, + actor_poll_duration: LabelGuardedGaugeVec<1>, + actor_poll_cnt: LabelGuardedIntGaugeVec<1>, + actor_idle_duration: LabelGuardedGaugeVec<1>, + actor_idle_cnt: LabelGuardedIntGaugeVec<1>, // Streaming actor - pub actor_memory_usage: GenericGaugeVec, - pub actor_in_record_cnt: LabelGuardedIntCounterVec<3>, + #[expect(dead_code)] + actor_memory_usage: LabelGuardedIntGaugeVec<2>, + actor_in_record_cnt: LabelGuardedIntCounterVec<3>, pub actor_out_record_cnt: LabelGuardedIntCounterVec<2>, // Source - pub source_output_row_count: GenericCounterVec, - pub source_split_change_count: GenericCounterVec, + pub source_output_row_count: LabelGuardedIntCounterVec<4>, + pub source_split_change_count: LabelGuardedIntCounterVec<4>, pub source_backfill_row_count: LabelGuardedIntCounterVec<4>, - // Sink & materialized view - pub sink_input_row_count: LabelGuardedIntCounterVec<3>, - pub mview_input_row_count: IntCounterVec, - pub sink_chunk_buffer_size: LabelGuardedIntGaugeVec<3>, + // Sink + sink_input_row_count: LabelGuardedIntCounterVec<3>, + sink_chunk_buffer_size: LabelGuardedIntGaugeVec<3>, // Exchange (see also `compute::ExchangeServiceMetrics`) - pub exchange_frag_recv_size: GenericCounterVec, + pub exchange_frag_recv_size: LabelGuardedIntCounterVec<2>, // Streaming Merge (We break out this metric from `barrier_align_duration` because // the alignment happens on different levels) @@ -81,12 +84,12 @@ pub struct StreamingMetrics { // Backpressure pub actor_output_buffer_blocking_duration_ns: LabelGuardedIntCounterVec<3>, - pub actor_input_buffer_blocking_duration_ns: LabelGuardedIntCounterVec<3>, + actor_input_buffer_blocking_duration_ns: LabelGuardedIntCounterVec<3>, // Streaming Join - pub join_lookup_miss_count: LabelGuardedIntCounterVec<5>, - pub join_lookup_total_count: LabelGuardedIntCounterVec<5>, - pub join_insert_cache_miss_count: LabelGuardedIntCounterVec<5>, + pub join_lookup_miss_count: LabelGuardedIntCounterVec<4>, + pub join_lookup_total_count: LabelGuardedIntCounterVec<4>, + pub join_insert_cache_miss_count: LabelGuardedIntCounterVec<4>, pub join_actor_input_waiting_duration_ns: LabelGuardedIntCounterVec<2>, pub join_match_duration_ns: LabelGuardedIntCounterVec<3>, pub join_cached_entry_count: LabelGuardedIntGaugeVec<3>, @@ -96,51 +99,52 @@ pub struct StreamingMetrics { pub barrier_align_duration: RelabeledGuardedHistogramVec<4>, // Streaming Aggregation - pub agg_lookup_miss_count: GenericCounterVec, - pub agg_total_lookup_count: GenericCounterVec, - pub agg_cached_entry_count: GenericGaugeVec, - pub agg_chunk_lookup_miss_count: GenericCounterVec, - pub agg_chunk_total_lookup_count: GenericCounterVec, - pub agg_distinct_cache_miss_count: GenericCounterVec, - pub agg_distinct_total_cache_count: GenericCounterVec, - pub agg_distinct_cached_entry_count: GenericGaugeVec, - pub agg_dirty_groups_count: GenericGaugeVec, - pub agg_dirty_groups_heap_size: GenericGaugeVec, + agg_lookup_miss_count: LabelGuardedIntCounterVec<3>, + agg_total_lookup_count: LabelGuardedIntCounterVec<3>, + agg_cached_entry_count: LabelGuardedIntGaugeVec<3>, + agg_chunk_lookup_miss_count: LabelGuardedIntCounterVec<3>, + agg_chunk_total_lookup_count: LabelGuardedIntCounterVec<3>, + agg_dirty_groups_count: LabelGuardedIntGaugeVec<3>, + agg_dirty_groups_heap_size: LabelGuardedIntGaugeVec<3>, + agg_distinct_cache_miss_count: LabelGuardedIntCounterVec<3>, + agg_distinct_total_cache_count: LabelGuardedIntCounterVec<3>, + agg_distinct_cached_entry_count: LabelGuardedIntGaugeVec<3>, // Streaming TopN - pub group_top_n_cache_miss_count: GenericCounterVec, - pub group_top_n_total_query_cache_count: GenericCounterVec, - pub group_top_n_cached_entry_count: GenericGaugeVec, - pub group_top_n_appendonly_cache_miss_count: GenericCounterVec, - pub group_top_n_appendonly_total_query_cache_count: GenericCounterVec, - pub group_top_n_appendonly_cached_entry_count: GenericGaugeVec, + group_top_n_cache_miss_count: LabelGuardedIntCounterVec<3>, + group_top_n_total_query_cache_count: LabelGuardedIntCounterVec<3>, + group_top_n_cached_entry_count: LabelGuardedIntGaugeVec<3>, + // TODO(rc): why not just use the above three? + group_top_n_appendonly_cache_miss_count: LabelGuardedIntCounterVec<3>, + group_top_n_appendonly_total_query_cache_count: LabelGuardedIntCounterVec<3>, + group_top_n_appendonly_cached_entry_count: LabelGuardedIntGaugeVec<3>, // Lookup executor - pub lookup_cache_miss_count: GenericCounterVec, - pub lookup_total_query_cache_count: GenericCounterVec, - pub lookup_cached_entry_count: GenericGaugeVec, + lookup_cache_miss_count: LabelGuardedIntCounterVec<3>, + lookup_total_query_cache_count: LabelGuardedIntCounterVec<3>, + lookup_cached_entry_count: LabelGuardedIntGaugeVec<3>, // temporal join - pub temporal_join_cache_miss_count: GenericCounterVec, - pub temporal_join_total_query_cache_count: GenericCounterVec, - pub temporal_join_cached_entry_count: GenericGaugeVec, + temporal_join_cache_miss_count: LabelGuardedIntCounterVec<3>, + temporal_join_total_query_cache_count: LabelGuardedIntCounterVec<3>, + temporal_join_cached_entry_count: LabelGuardedIntGaugeVec<3>, // Backfill - pub backfill_snapshot_read_row_count: LabelGuardedIntCounterVec<2>, - pub backfill_upstream_output_row_count: LabelGuardedIntCounterVec<2>, + backfill_snapshot_read_row_count: LabelGuardedIntCounterVec<2>, + backfill_upstream_output_row_count: LabelGuardedIntCounterVec<2>, // CDC Backfill - pub cdc_backfill_snapshot_read_row_count: GenericCounterVec, - pub cdc_backfill_upstream_output_row_count: GenericCounterVec, + cdc_backfill_snapshot_read_row_count: LabelGuardedIntCounterVec<2>, + cdc_backfill_upstream_output_row_count: LabelGuardedIntCounterVec<2>, // Over Window - pub over_window_cached_entry_count: GenericGaugeVec, - pub over_window_cache_lookup_count: GenericCounterVec, - pub over_window_cache_miss_count: GenericCounterVec, - pub over_window_range_cache_entry_count: GenericGaugeVec, - pub over_window_range_cache_lookup_count: GenericCounterVec, - pub over_window_range_cache_left_miss_count: GenericCounterVec, - pub over_window_range_cache_right_miss_count: GenericCounterVec, + over_window_cached_entry_count: LabelGuardedIntGaugeVec<3>, + over_window_cache_lookup_count: LabelGuardedIntCounterVec<3>, + over_window_cache_miss_count: LabelGuardedIntCounterVec<3>, + over_window_range_cache_entry_count: LabelGuardedIntGaugeVec<3>, + over_window_range_cache_lookup_count: LabelGuardedIntCounterVec<3>, + over_window_range_cache_left_miss_count: LabelGuardedIntCounterVec<3>, + over_window_range_cache_right_miss_count: LabelGuardedIntCounterVec<3>, /// The duration from receipt of barrier to all actors collection. /// And the max of all node `barrier_inflight_latency` is the latency for a barrier @@ -152,14 +156,14 @@ pub struct StreamingMetrics { pub barrier_manager_progress: IntCounter, // Sink related metrics - pub sink_commit_duration: LabelGuardedHistogramVec<3>, - pub connector_sink_rows_received: LabelGuardedIntCounterVec<2>, - pub log_store_first_write_epoch: LabelGuardedIntGaugeVec<3>, - pub log_store_latest_write_epoch: LabelGuardedIntGaugeVec<3>, - pub log_store_write_rows: LabelGuardedIntCounterVec<3>, - pub log_store_latest_read_epoch: LabelGuardedIntGaugeVec<3>, - pub log_store_read_rows: LabelGuardedIntCounterVec<3>, - pub log_store_reader_wait_new_future_duration_ns: LabelGuardedIntCounterVec<3>, + sink_commit_duration: LabelGuardedHistogramVec<3>, + connector_sink_rows_received: LabelGuardedIntCounterVec<2>, + log_store_first_write_epoch: LabelGuardedIntGaugeVec<3>, + log_store_latest_write_epoch: LabelGuardedIntGaugeVec<3>, + log_store_write_rows: LabelGuardedIntCounterVec<3>, + log_store_latest_read_epoch: LabelGuardedIntGaugeVec<3>, + log_store_read_rows: LabelGuardedIntCounterVec<3>, + log_store_reader_wait_new_future_duration_ns: LabelGuardedIntCounterVec<3>, pub kv_log_store_storage_write_count: LabelGuardedIntCounterVec<3>, pub kv_log_store_storage_write_size: LabelGuardedIntCounterVec<3>, pub kv_log_store_rewind_count: LabelGuardedIntCounterVec<3>, @@ -172,11 +176,11 @@ pub struct StreamingMetrics { pub kv_log_store_buffer_unconsumed_min_epoch: LabelGuardedIntGaugeVec<3>, // Sink iceberg metrics - pub iceberg_write_qps: LabelGuardedIntCounterVec<2>, - pub iceberg_write_latency: LabelGuardedHistogramVec<2>, - pub iceberg_rolling_unflushed_data_file: LabelGuardedIntGaugeVec<2>, - pub iceberg_position_delete_cache_num: LabelGuardedIntGaugeVec<2>, - pub iceberg_partition_num: LabelGuardedIntGaugeVec<2>, + iceberg_write_qps: LabelGuardedIntCounterVec<2>, + iceberg_write_latency: LabelGuardedHistogramVec<2>, + iceberg_rolling_unflushed_data_file: LabelGuardedIntGaugeVec<2>, + iceberg_position_delete_cache_num: LabelGuardedIntGaugeVec<2>, + iceberg_partition_num: LabelGuardedIntGaugeVec<2>, // Memory management pub lru_runtime_loop_count: IntCounter, @@ -189,13 +193,12 @@ pub struct StreamingMetrics { pub jemalloc_metadata_bytes: IntGauge, pub jvm_allocated_bytes: IntGauge, pub jvm_active_bytes: IntGauge, - - // Materialize - pub materialize_cache_hit_count: GenericCounterVec, - pub materialize_cache_total_count: GenericCounterVec, - - // Memory pub stream_memory_usage: LabelGuardedIntGaugeVec<3>, + + // Materialized view + materialize_cache_hit_count: LabelGuardedIntCounterVec<3>, + materialize_cache_total_count: LabelGuardedIntCounterVec<3>, + materialize_input_row_count: LabelGuardedIntCounterVec<3>, } pub static GLOBAL_STREAMING_METRICS: OnceLock = OnceLock::new(); @@ -208,7 +211,7 @@ pub fn global_streaming_metrics(metric_level: MetricLevel) -> StreamingMetrics { impl StreamingMetrics { fn new(registry: &Registry, level: MetricLevel) -> Self { - let executor_row_count = register_int_counter_vec_with_registry!( + let executor_row_count = register_guarded_int_counter_vec_with_registry!( "stream_executor_row_count", "Total number of rows that have been output from each executor", &["actor_id", "fragment_id", "executor_identity"], @@ -216,7 +219,7 @@ impl StreamingMetrics { ) .unwrap(); - let source_output_row_count = register_int_counter_vec_with_registry!( + let source_output_row_count = register_guarded_int_counter_vec_with_registry!( "stream_source_output_rows_counts", "Total number of rows that have been output from source", &["source_id", "source_name", "actor_id", "fragment_id"], @@ -224,7 +227,7 @@ impl StreamingMetrics { ) .unwrap(); - let source_split_change_count = register_int_counter_vec_with_registry!( + let source_split_change_count = register_guarded_int_counter_vec_with_registry!( "stream_source_split_change_event_count", "Total number of split change events that have been operated by source", &["source_id", "source_name", "actor_id", "fragment_id"], @@ -248,7 +251,7 @@ impl StreamingMetrics { ) .unwrap(); - let mview_input_row_count = register_int_counter_vec_with_registry!( + let materialize_input_row_count = register_guarded_int_counter_vec_with_registry!( "stream_mview_input_row_count", "Total number of rows streamed into materialize executors", &["table_id", "actor_id", "fragment_id"], @@ -264,7 +267,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_execution_time = register_gauge_vec_with_registry!( + let actor_execution_time = register_guarded_gauge_vec_with_registry!( "stream_actor_actor_execution_time", "Total execution time (s) of an actor", &["actor_id"], @@ -290,7 +293,7 @@ impl StreamingMetrics { ) .unwrap(); - let exchange_frag_recv_size = register_int_counter_vec_with_registry!( + let exchange_frag_recv_size = register_guarded_int_counter_vec_with_registry!( "stream_exchange_frag_recv_size", "Total size of messages that have been received from upstream Fragment", &["up_fragment_id", "down_fragment_id"], @@ -298,7 +301,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_fast_poll_duration = register_gauge_vec_with_registry!( + let actor_fast_poll_duration = register_guarded_gauge_vec_with_registry!( "stream_actor_fast_poll_duration", "tokio's metrics", &["actor_id"], @@ -306,7 +309,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_fast_poll_cnt = register_int_gauge_vec_with_registry!( + let actor_fast_poll_cnt = register_guarded_int_gauge_vec_with_registry!( "stream_actor_fast_poll_cnt", "tokio's metrics", &["actor_id"], @@ -314,7 +317,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_slow_poll_duration = register_gauge_vec_with_registry!( + let actor_slow_poll_duration = register_guarded_gauge_vec_with_registry!( "stream_actor_slow_poll_duration", "tokio's metrics", &["actor_id"], @@ -322,7 +325,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_slow_poll_cnt = register_int_gauge_vec_with_registry!( + let actor_slow_poll_cnt = register_guarded_int_gauge_vec_with_registry!( "stream_actor_slow_poll_cnt", "tokio's metrics", &["actor_id"], @@ -330,7 +333,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_poll_duration = register_gauge_vec_with_registry!( + let actor_poll_duration = register_guarded_gauge_vec_with_registry!( "stream_actor_poll_duration", "tokio's metrics", &["actor_id"], @@ -338,7 +341,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_poll_cnt = register_int_gauge_vec_with_registry!( + let actor_poll_cnt = register_guarded_int_gauge_vec_with_registry!( "stream_actor_poll_cnt", "tokio's metrics", &["actor_id"], @@ -346,7 +349,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_scheduled_duration = register_gauge_vec_with_registry!( + let actor_scheduled_duration = register_guarded_gauge_vec_with_registry!( "stream_actor_scheduled_duration", "tokio's metrics", &["actor_id"], @@ -354,7 +357,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_scheduled_cnt = register_int_gauge_vec_with_registry!( + let actor_scheduled_cnt = register_guarded_int_gauge_vec_with_registry!( "stream_actor_scheduled_cnt", "tokio's metrics", &["actor_id"], @@ -362,7 +365,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_idle_duration = register_gauge_vec_with_registry!( + let actor_idle_duration = register_guarded_gauge_vec_with_registry!( "stream_actor_idle_duration", "tokio's metrics", &["actor_id"], @@ -370,7 +373,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_idle_cnt = register_int_gauge_vec_with_registry!( + let actor_idle_cnt = register_guarded_int_gauge_vec_with_registry!( "stream_actor_idle_cnt", "tokio's metrics", &["actor_id"], @@ -394,7 +397,7 @@ impl StreamingMetrics { ) .unwrap(); - let actor_memory_usage = register_int_gauge_vec_with_registry!( + let actor_memory_usage = register_guarded_int_gauge_vec_with_registry!( "actor_memory_usage", "Memory usage (bytes)", &["actor_id", "fragment_id"], @@ -425,13 +428,7 @@ impl StreamingMetrics { let join_lookup_miss_count = register_guarded_int_counter_vec_with_registry!( "stream_join_lookup_miss_count", "Join executor lookup miss duration", - &[ - "side", - "join_table_id", - "degree_table_id", - "actor_id", - "fragment_id" - ], + &["side", "join_table_id", "actor_id", "fragment_id"], registry ) .unwrap(); @@ -439,13 +436,7 @@ impl StreamingMetrics { let join_lookup_total_count = register_guarded_int_counter_vec_with_registry!( "stream_join_lookup_total_count", "Join executor lookup total operation", - &[ - "side", - "join_table_id", - "degree_table_id", - "actor_id", - "fragment_id" - ], + &["side", "join_table_id", "actor_id", "fragment_id"], registry ) .unwrap(); @@ -453,13 +444,7 @@ impl StreamingMetrics { let join_insert_cache_miss_count = register_guarded_int_counter_vec_with_registry!( "stream_join_insert_cache_miss_count", "Join executor cache miss when insert operation", - &[ - "side", - "join_table_id", - "degree_table_id", - "actor_id", - "fragment_id" - ], + &["side", "join_table_id", "actor_id", "fragment_id"], registry ) .unwrap(); @@ -527,7 +512,7 @@ impl StreamingMetrics { 1, ); - let agg_lookup_miss_count = register_int_counter_vec_with_registry!( + let agg_lookup_miss_count = register_guarded_int_counter_vec_with_registry!( "stream_agg_lookup_miss_count", "Aggregation executor lookup miss duration", &["table_id", "actor_id", "fragment_id"], @@ -535,7 +520,7 @@ impl StreamingMetrics { ) .unwrap(); - let agg_total_lookup_count = register_int_counter_vec_with_registry!( + let agg_total_lookup_count = register_guarded_int_counter_vec_with_registry!( "stream_agg_lookup_total_count", "Aggregation executor lookup total operation", &["table_id", "actor_id", "fragment_id"], @@ -543,7 +528,7 @@ impl StreamingMetrics { ) .unwrap(); - let agg_distinct_cache_miss_count = register_int_counter_vec_with_registry!( + let agg_distinct_cache_miss_count = register_guarded_int_counter_vec_with_registry!( "stream_agg_distinct_cache_miss_count", "Aggregation executor dinsinct miss duration", &["table_id", "actor_id", "fragment_id"], @@ -551,7 +536,7 @@ impl StreamingMetrics { ) .unwrap(); - let agg_distinct_total_cache_count = register_int_counter_vec_with_registry!( + let agg_distinct_total_cache_count = register_guarded_int_counter_vec_with_registry!( "stream_agg_distinct_total_cache_count", "Aggregation executor distinct total operation", &["table_id", "actor_id", "fragment_id"], @@ -559,7 +544,7 @@ impl StreamingMetrics { ) .unwrap(); - let agg_distinct_cached_entry_count = register_int_gauge_vec_with_registry!( + let agg_distinct_cached_entry_count = register_guarded_int_gauge_vec_with_registry!( "stream_agg_distinct_cached_entry_count", "Total entry counts in distinct aggregation executor cache", &["table_id", "actor_id", "fragment_id"], @@ -567,7 +552,7 @@ impl StreamingMetrics { ) .unwrap(); - let agg_dirty_groups_count = register_int_gauge_vec_with_registry!( + let agg_dirty_groups_count = register_guarded_int_gauge_vec_with_registry!( "stream_agg_dirty_groups_count", "Total dirty group counts in aggregation executor", &["table_id", "actor_id", "fragment_id"], @@ -575,7 +560,7 @@ impl StreamingMetrics { ) .unwrap(); - let agg_dirty_groups_heap_size = register_int_gauge_vec_with_registry!( + let agg_dirty_groups_heap_size = register_guarded_int_gauge_vec_with_registry!( "stream_agg_dirty_groups_heap_size", "Total dirty group heap size in aggregation executor", &["table_id", "actor_id", "fragment_id"], @@ -583,7 +568,7 @@ impl StreamingMetrics { ) .unwrap(); - let group_top_n_cache_miss_count = register_int_counter_vec_with_registry!( + let group_top_n_cache_miss_count = register_guarded_int_counter_vec_with_registry!( "stream_group_top_n_cache_miss_count", "Group top n executor cache miss count", &["table_id", "actor_id", "fragment_id"], @@ -591,7 +576,7 @@ impl StreamingMetrics { ) .unwrap(); - let group_top_n_total_query_cache_count = register_int_counter_vec_with_registry!( + let group_top_n_total_query_cache_count = register_guarded_int_counter_vec_with_registry!( "stream_group_top_n_total_query_cache_count", "Group top n executor query cache total count", &["table_id", "actor_id", "fragment_id"], @@ -599,7 +584,7 @@ impl StreamingMetrics { ) .unwrap(); - let group_top_n_cached_entry_count = register_int_gauge_vec_with_registry!( + let group_top_n_cached_entry_count = register_guarded_int_gauge_vec_with_registry!( "stream_group_top_n_cached_entry_count", "Total entry counts in group top n executor cache", &["table_id", "actor_id", "fragment_id"], @@ -607,16 +592,17 @@ impl StreamingMetrics { ) .unwrap(); - let group_top_n_appendonly_cache_miss_count = register_int_counter_vec_with_registry!( - "stream_group_top_n_appendonly_cache_miss_count", - "Group top n appendonly executor cache miss count", - &["table_id", "actor_id", "fragment_id"], - registry - ) - .unwrap(); + let group_top_n_appendonly_cache_miss_count = + register_guarded_int_counter_vec_with_registry!( + "stream_group_top_n_appendonly_cache_miss_count", + "Group top n appendonly executor cache miss count", + &["table_id", "actor_id", "fragment_id"], + registry + ) + .unwrap(); let group_top_n_appendonly_total_query_cache_count = - register_int_counter_vec_with_registry!( + register_guarded_int_counter_vec_with_registry!( "stream_group_top_n_appendonly_total_query_cache_count", "Group top n appendonly executor total cache count", &["table_id", "actor_id", "fragment_id"], @@ -624,15 +610,16 @@ impl StreamingMetrics { ) .unwrap(); - let group_top_n_appendonly_cached_entry_count = register_int_gauge_vec_with_registry!( - "stream_group_top_n_appendonly_cached_entry_count", - "Total entry counts in group top n appendonly executor cache", - &["table_id", "actor_id", "fragment_id"], - registry - ) - .unwrap(); + let group_top_n_appendonly_cached_entry_count = + register_guarded_int_gauge_vec_with_registry!( + "stream_group_top_n_appendonly_cached_entry_count", + "Total entry counts in group top n appendonly executor cache", + &["table_id", "actor_id", "fragment_id"], + registry + ) + .unwrap(); - let lookup_cache_miss_count = register_int_counter_vec_with_registry!( + let lookup_cache_miss_count = register_guarded_int_counter_vec_with_registry!( "stream_lookup_cache_miss_count", "Lookup executor cache miss count", &["table_id", "actor_id", "fragment_id"], @@ -640,7 +627,7 @@ impl StreamingMetrics { ) .unwrap(); - let lookup_total_query_cache_count = register_int_counter_vec_with_registry!( + let lookup_total_query_cache_count = register_guarded_int_counter_vec_with_registry!( "stream_lookup_total_query_cache_count", "Lookup executor query cache total count", &["table_id", "actor_id", "fragment_id"], @@ -648,7 +635,7 @@ impl StreamingMetrics { ) .unwrap(); - let lookup_cached_entry_count = register_int_gauge_vec_with_registry!( + let lookup_cached_entry_count = register_guarded_int_gauge_vec_with_registry!( "stream_lookup_cached_entry_count", "Total entry counts in lookup executor cache", &["table_id", "actor_id", "fragment_id"], @@ -656,7 +643,7 @@ impl StreamingMetrics { ) .unwrap(); - let temporal_join_cache_miss_count = register_int_counter_vec_with_registry!( + let temporal_join_cache_miss_count = register_guarded_int_counter_vec_with_registry!( "stream_temporal_join_cache_miss_count", "Temporal join executor cache miss count", &["table_id", "actor_id", "fragment_id"], @@ -664,15 +651,16 @@ impl StreamingMetrics { ) .unwrap(); - let temporal_join_total_query_cache_count = register_int_counter_vec_with_registry!( - "stream_temporal_join_total_query_cache_count", - "Temporal join executor query cache total count", - &["table_id", "actor_id", "fragment_id"], - registry - ) - .unwrap(); + let temporal_join_total_query_cache_count = + register_guarded_int_counter_vec_with_registry!( + "stream_temporal_join_total_query_cache_count", + "Temporal join executor query cache total count", + &["table_id", "actor_id", "fragment_id"], + registry + ) + .unwrap(); - let temporal_join_cached_entry_count = register_int_gauge_vec_with_registry!( + let temporal_join_cached_entry_count = register_guarded_int_gauge_vec_with_registry!( "stream_temporal_join_cached_entry_count", "Total entry count in temporal join executor cache", &["table_id", "actor_id", "fragment_id"], @@ -680,7 +668,7 @@ impl StreamingMetrics { ) .unwrap(); - let agg_cached_entry_count = register_int_gauge_vec_with_registry!( + let agg_cached_entry_count = register_guarded_int_gauge_vec_with_registry!( "stream_agg_cached_entry_count", "Number of cached keys in streaming aggregation operators", &["table_id", "actor_id", "fragment_id"], @@ -688,7 +676,7 @@ impl StreamingMetrics { ) .unwrap(); - let agg_chunk_lookup_miss_count = register_int_counter_vec_with_registry!( + let agg_chunk_lookup_miss_count = register_guarded_int_counter_vec_with_registry!( "stream_agg_chunk_lookup_miss_count", "Aggregation executor chunk-level lookup miss duration", &["table_id", "actor_id", "fragment_id"], @@ -696,7 +684,7 @@ impl StreamingMetrics { ) .unwrap(); - let agg_chunk_total_lookup_count = register_int_counter_vec_with_registry!( + let agg_chunk_total_lookup_count = register_guarded_int_counter_vec_with_registry!( "stream_agg_chunk_lookup_total_count", "Aggregation executor chunk-level lookup total operation", &["table_id", "actor_id", "fragment_id"], @@ -720,7 +708,7 @@ impl StreamingMetrics { ) .unwrap(); - let cdc_backfill_snapshot_read_row_count = register_int_counter_vec_with_registry!( + let cdc_backfill_snapshot_read_row_count = register_guarded_int_counter_vec_with_registry!( "stream_cdc_backfill_snapshot_read_row_count", "Total number of rows that have been read from the cdc_backfill snapshot", &["table_id", "actor_id"], @@ -728,15 +716,16 @@ impl StreamingMetrics { ) .unwrap(); - let cdc_backfill_upstream_output_row_count = register_int_counter_vec_with_registry!( - "stream_cdc_backfill_upstream_output_row_count", - "Total number of rows that have been output from the cdc_backfill upstream", - &["table_id", "actor_id"], - registry - ) - .unwrap(); + let cdc_backfill_upstream_output_row_count = + register_guarded_int_counter_vec_with_registry!( + "stream_cdc_backfill_upstream_output_row_count", + "Total number of rows that have been output from the cdc_backfill upstream", + &["table_id", "actor_id"], + registry + ) + .unwrap(); - let over_window_cached_entry_count = register_int_gauge_vec_with_registry!( + let over_window_cached_entry_count = register_guarded_int_gauge_vec_with_registry!( "stream_over_window_cached_entry_count", "Total entry (partition) count in over window executor cache", &["table_id", "actor_id", "fragment_id"], @@ -744,7 +733,7 @@ impl StreamingMetrics { ) .unwrap(); - let over_window_cache_lookup_count = register_int_counter_vec_with_registry!( + let over_window_cache_lookup_count = register_guarded_int_counter_vec_with_registry!( "stream_over_window_cache_lookup_count", "Over window executor cache lookup count", &["table_id", "actor_id", "fragment_id"], @@ -752,7 +741,7 @@ impl StreamingMetrics { ) .unwrap(); - let over_window_cache_miss_count = register_int_counter_vec_with_registry!( + let over_window_cache_miss_count = register_guarded_int_counter_vec_with_registry!( "stream_over_window_cache_miss_count", "Over window executor cache miss count", &["table_id", "actor_id", "fragment_id"], @@ -760,7 +749,7 @@ impl StreamingMetrics { ) .unwrap(); - let over_window_range_cache_entry_count = register_int_gauge_vec_with_registry!( + let over_window_range_cache_entry_count = register_guarded_int_gauge_vec_with_registry!( "stream_over_window_range_cache_entry_count", "Over window partition range cache entry count", &["table_id", "actor_id", "fragment_id"], @@ -768,7 +757,7 @@ impl StreamingMetrics { ) .unwrap(); - let over_window_range_cache_lookup_count = register_int_counter_vec_with_registry!( + let over_window_range_cache_lookup_count = register_guarded_int_counter_vec_with_registry!( "stream_over_window_range_cache_lookup_count", "Over window partition range cache lookup count", &["table_id", "actor_id", "fragment_id"], @@ -776,21 +765,23 @@ impl StreamingMetrics { ) .unwrap(); - let over_window_range_cache_left_miss_count = register_int_counter_vec_with_registry!( - "stream_over_window_range_cache_left_miss_count", - "Over window partition range cache left miss count", - &["table_id", "actor_id", "fragment_id"], - registry - ) - .unwrap(); + let over_window_range_cache_left_miss_count = + register_guarded_int_counter_vec_with_registry!( + "stream_over_window_range_cache_left_miss_count", + "Over window partition range cache left miss count", + &["table_id", "actor_id", "fragment_id"], + registry + ) + .unwrap(); - let over_window_range_cache_right_miss_count = register_int_counter_vec_with_registry!( - "stream_over_window_range_cache_right_miss_count", - "Over window partition range cache right miss count", - &["table_id", "actor_id", "fragment_id"], - registry - ) - .unwrap(); + let over_window_range_cache_right_miss_count = + register_guarded_int_counter_vec_with_registry!( + "stream_over_window_range_cache_right_miss_count", + "Over window partition range cache right miss count", + &["table_id", "actor_id", "fragment_id"], + registry + ) + .unwrap(); let opts = histogram_opts!( "stream_barrier_inflight_duration_seconds", @@ -1049,18 +1040,18 @@ impl StreamingMetrics { ) .unwrap(); - let materialize_cache_hit_count = register_int_counter_vec_with_registry!( + let materialize_cache_hit_count = register_guarded_int_counter_vec_with_registry!( "stream_materialize_cache_hit_count", "Materialize executor cache hit count", - &["table_id", "actor_id"], + &["table_id", "actor_id", "fragment_id"], registry ) .unwrap(); - let materialize_cache_total_count = register_int_counter_vec_with_registry!( + let materialize_cache_total_count = register_guarded_int_counter_vec_with_registry!( "stream_materialize_cache_total_count", "Materialize executor cache total operation", - &["table_id", "actor_id"], + &["table_id", "actor_id", "fragment_id"], registry ) .unwrap(); @@ -1134,7 +1125,6 @@ impl StreamingMetrics { source_split_change_count, source_backfill_row_count, sink_input_row_count, - mview_input_row_count, sink_chunk_buffer_size, exchange_frag_recv_size, merge_barrier_align_duration, @@ -1153,11 +1143,11 @@ impl StreamingMetrics { agg_cached_entry_count, agg_chunk_lookup_miss_count, agg_chunk_total_lookup_count, + agg_dirty_groups_count, + agg_dirty_groups_heap_size, agg_distinct_cache_miss_count, agg_distinct_total_cache_count, agg_distinct_cached_entry_count, - agg_dirty_groups_count, - agg_dirty_groups_heap_size, group_top_n_cache_miss_count, group_top_n_total_query_cache_count, group_top_n_cached_entry_count, @@ -1217,9 +1207,10 @@ impl StreamingMetrics { jemalloc_metadata_bytes, jvm_allocated_bytes, jvm_active_bytes, + stream_memory_usage, materialize_cache_hit_count, materialize_cache_total_count, - stream_memory_usage, + materialize_input_row_count, } } @@ -1297,4 +1288,428 @@ impl StreamingMetrics { iceberg_partition_num, } } + + pub fn new_actor_metrics(&self, actor_id: ActorId) -> ActorMetrics { + let label_list: &[&str; 1] = &[&actor_id.to_string()]; + let actor_execution_time = self + .actor_execution_time + .with_guarded_label_values(label_list); + let actor_scheduled_duration = self + .actor_scheduled_duration + .with_guarded_label_values(label_list); + let actor_scheduled_cnt = self + .actor_scheduled_cnt + .with_guarded_label_values(label_list); + let actor_fast_poll_duration = self + .actor_fast_poll_duration + .with_guarded_label_values(label_list); + let actor_fast_poll_cnt = self + .actor_fast_poll_cnt + .with_guarded_label_values(label_list); + let actor_slow_poll_duration = self + .actor_slow_poll_duration + .with_guarded_label_values(label_list); + let actor_slow_poll_cnt = self + .actor_slow_poll_cnt + .with_guarded_label_values(label_list); + let actor_poll_duration = self + .actor_poll_duration + .with_guarded_label_values(label_list); + let actor_poll_cnt = self.actor_poll_cnt.with_guarded_label_values(label_list); + let actor_idle_duration = self + .actor_idle_duration + .with_guarded_label_values(label_list); + let actor_idle_cnt = self.actor_idle_cnt.with_guarded_label_values(label_list); + ActorMetrics { + actor_execution_time, + actor_scheduled_duration, + actor_scheduled_cnt, + actor_fast_poll_duration, + actor_fast_poll_cnt, + actor_slow_poll_duration, + actor_slow_poll_cnt, + actor_poll_duration, + actor_poll_cnt, + actor_idle_duration, + actor_idle_cnt, + } + } + + pub(crate) fn new_actor_input_metrics( + &self, + actor_id: ActorId, + fragment_id: FragmentId, + upstream_fragment_id: FragmentId, + ) -> ActorInputMetrics { + let actor_id_str = actor_id.to_string(); + let fragment_id_str = fragment_id.to_string(); + let upstream_fragment_id_str = upstream_fragment_id.to_string(); + ActorInputMetrics { + actor_in_record_cnt: self.actor_in_record_cnt.with_guarded_label_values(&[ + &actor_id_str, + &fragment_id_str, + &upstream_fragment_id_str, + ]), + actor_input_buffer_blocking_duration_ns: self + .actor_input_buffer_blocking_duration_ns + .with_guarded_label_values(&[ + &actor_id_str, + &fragment_id_str, + &upstream_fragment_id_str, + ]), + } + } + + pub fn new_sink_exec_metrics( + &self, + id: SinkId, + actor_id: ActorId, + fragment_id: FragmentId, + ) -> SinkExecutorMetrics { + let label_list: &[&str; 3] = &[ + &id.to_string(), + &actor_id.to_string(), + &fragment_id.to_string(), + ]; + SinkExecutorMetrics { + sink_input_row_count: self + .sink_input_row_count + .with_guarded_label_values(label_list), + sink_chunk_buffer_size: self + .sink_chunk_buffer_size + .with_guarded_label_values(label_list), + } + } + + pub fn new_group_top_n_metrics( + &self, + table_id: u32, + actor_id: ActorId, + fragment_id: FragmentId, + ) -> GroupTopNMetrics { + let label_list: &[&str; 3] = &[ + &table_id.to_string(), + &actor_id.to_string(), + &fragment_id.to_string(), + ]; + + GroupTopNMetrics { + group_top_n_cache_miss_count: self + .group_top_n_cache_miss_count + .with_guarded_label_values(label_list), + group_top_n_total_query_cache_count: self + .group_top_n_total_query_cache_count + .with_guarded_label_values(label_list), + group_top_n_cached_entry_count: self + .group_top_n_cached_entry_count + .with_guarded_label_values(label_list), + } + } + + pub fn new_append_only_group_top_n_metrics( + &self, + table_id: u32, + actor_id: ActorId, + fragment_id: FragmentId, + ) -> GroupTopNMetrics { + let label_list: &[&str; 3] = &[ + &table_id.to_string(), + &actor_id.to_string(), + &fragment_id.to_string(), + ]; + + GroupTopNMetrics { + group_top_n_cache_miss_count: self + .group_top_n_appendonly_cache_miss_count + .with_guarded_label_values(label_list), + group_top_n_total_query_cache_count: self + .group_top_n_appendonly_total_query_cache_count + .with_guarded_label_values(label_list), + group_top_n_cached_entry_count: self + .group_top_n_appendonly_cached_entry_count + .with_guarded_label_values(label_list), + } + } + + pub fn new_lookup_executor_metrics( + &self, + table_id: TableId, + actor_id: ActorId, + fragment_id: FragmentId, + ) -> LookupExecutorMetrics { + let label_list: &[&str; 3] = &[ + &table_id.to_string(), + &actor_id.to_string(), + &fragment_id.to_string(), + ]; + + LookupExecutorMetrics { + lookup_cache_miss_count: self + .lookup_cache_miss_count + .with_guarded_label_values(label_list), + lookup_total_query_cache_count: self + .lookup_total_query_cache_count + .with_guarded_label_values(label_list), + lookup_cached_entry_count: self + .lookup_cached_entry_count + .with_guarded_label_values(label_list), + } + } + + pub fn new_hash_agg_metrics( + &self, + table_id: u32, + actor_id: ActorId, + fragment_id: FragmentId, + ) -> HashAggMetrics { + let label_list: &[&str; 3] = &[ + &table_id.to_string(), + &actor_id.to_string(), + &fragment_id.to_string(), + ]; + HashAggMetrics { + agg_lookup_miss_count: self + .agg_lookup_miss_count + .with_guarded_label_values(label_list), + agg_total_lookup_count: self + .agg_total_lookup_count + .with_guarded_label_values(label_list), + agg_cached_entry_count: self + .agg_cached_entry_count + .with_guarded_label_values(label_list), + agg_chunk_lookup_miss_count: self + .agg_chunk_lookup_miss_count + .with_guarded_label_values(label_list), + agg_chunk_total_lookup_count: self + .agg_chunk_total_lookup_count + .with_guarded_label_values(label_list), + agg_dirty_groups_count: self + .agg_dirty_groups_count + .with_guarded_label_values(label_list), + agg_dirty_groups_heap_size: self + .agg_dirty_groups_heap_size + .with_guarded_label_values(label_list), + } + } + + pub fn new_agg_distinct_dedup_metrics( + &self, + table_id: u32, + actor_id: ActorId, + fragment_id: FragmentId, + ) -> AggDistinctDedupMetrics { + let label_list: &[&str; 3] = &[ + &table_id.to_string(), + &actor_id.to_string(), + &fragment_id.to_string(), + ]; + AggDistinctDedupMetrics { + agg_distinct_cache_miss_count: self + .agg_distinct_cache_miss_count + .with_guarded_label_values(label_list), + agg_distinct_total_cache_count: self + .agg_distinct_total_cache_count + .with_guarded_label_values(label_list), + agg_distinct_cached_entry_count: self + .agg_distinct_cached_entry_count + .with_guarded_label_values(label_list), + } + } + + pub fn new_temporal_join_metrics( + &self, + table_id: TableId, + actor_id: ActorId, + fragment_id: FragmentId, + ) -> TemporalJoinMetrics { + let label_list: &[&str; 3] = &[ + &table_id.to_string(), + &actor_id.to_string(), + &fragment_id.to_string(), + ]; + TemporalJoinMetrics { + temporal_join_cache_miss_count: self + .temporal_join_cache_miss_count + .with_guarded_label_values(label_list), + temporal_join_total_query_cache_count: self + .temporal_join_total_query_cache_count + .with_guarded_label_values(label_list), + temporal_join_cached_entry_count: self + .temporal_join_cached_entry_count + .with_guarded_label_values(label_list), + } + } + + pub fn new_backfill_metrics(&self, table_id: u32, actor_id: ActorId) -> BackfillMetrics { + let label_list: &[&str; 2] = &[&table_id.to_string(), &actor_id.to_string()]; + BackfillMetrics { + backfill_snapshot_read_row_count: self + .backfill_snapshot_read_row_count + .with_guarded_label_values(label_list), + backfill_upstream_output_row_count: self + .backfill_upstream_output_row_count + .with_guarded_label_values(label_list), + } + } + + pub fn new_cdc_backfill_metrics( + &self, + table_id: TableId, + actor_id: ActorId, + ) -> CdcBackfillMetrics { + let label_list: &[&str; 2] = &[&table_id.to_string(), &actor_id.to_string()]; + CdcBackfillMetrics { + cdc_backfill_snapshot_read_row_count: self + .cdc_backfill_snapshot_read_row_count + .with_guarded_label_values(label_list), + cdc_backfill_upstream_output_row_count: self + .cdc_backfill_upstream_output_row_count + .with_guarded_label_values(label_list), + } + } + + pub fn new_over_window_metrics( + &self, + table_id: u32, + actor_id: ActorId, + fragment_id: FragmentId, + ) -> OverWindowMetrics { + let label_list: &[&str; 3] = &[ + &table_id.to_string(), + &actor_id.to_string(), + &fragment_id.to_string(), + ]; + OverWindowMetrics { + over_window_cached_entry_count: self + .over_window_cached_entry_count + .with_guarded_label_values(label_list), + over_window_cache_lookup_count: self + .over_window_cache_lookup_count + .with_guarded_label_values(label_list), + over_window_cache_miss_count: self + .over_window_cache_miss_count + .with_guarded_label_values(label_list), + over_window_range_cache_entry_count: self + .over_window_range_cache_entry_count + .with_guarded_label_values(label_list), + over_window_range_cache_lookup_count: self + .over_window_range_cache_lookup_count + .with_guarded_label_values(label_list), + over_window_range_cache_left_miss_count: self + .over_window_range_cache_left_miss_count + .with_guarded_label_values(label_list), + over_window_range_cache_right_miss_count: self + .over_window_range_cache_right_miss_count + .with_guarded_label_values(label_list), + } + } + + pub fn new_materialize_metrics( + &self, + table_id: TableId, + actor_id: ActorId, + fragment_id: FragmentId, + ) -> MaterializeMetrics { + let label_list: &[&str; 3] = &[ + &table_id.to_string(), + &actor_id.to_string(), + &fragment_id.to_string(), + ]; + MaterializeMetrics { + materialize_cache_hit_count: self + .materialize_cache_hit_count + .with_guarded_label_values(label_list), + materialize_cache_total_count: self + .materialize_cache_total_count + .with_guarded_label_values(label_list), + materialize_input_row_count: self + .materialize_input_row_count + .with_guarded_label_values(label_list), + } + } +} + +pub(crate) struct ActorInputMetrics { + pub(crate) actor_in_record_cnt: LabelGuardedIntCounter<3>, + pub(crate) actor_input_buffer_blocking_duration_ns: LabelGuardedIntCounter<3>, +} + +/// Tokio metrics for actors +pub struct ActorMetrics { + pub actor_execution_time: LabelGuardedGauge<1>, + pub actor_scheduled_duration: LabelGuardedGauge<1>, + pub actor_scheduled_cnt: LabelGuardedIntGauge<1>, + pub actor_fast_poll_duration: LabelGuardedGauge<1>, + pub actor_fast_poll_cnt: LabelGuardedIntGauge<1>, + pub actor_slow_poll_duration: LabelGuardedGauge<1>, + pub actor_slow_poll_cnt: LabelGuardedIntGauge<1>, + pub actor_poll_duration: LabelGuardedGauge<1>, + pub actor_poll_cnt: LabelGuardedIntGauge<1>, + pub actor_idle_duration: LabelGuardedGauge<1>, + pub actor_idle_cnt: LabelGuardedIntGauge<1>, +} + +pub struct SinkExecutorMetrics { + pub sink_input_row_count: LabelGuardedIntCounter<3>, + pub sink_chunk_buffer_size: LabelGuardedIntGauge<3>, +} + +pub struct MaterializeMetrics { + pub materialize_cache_hit_count: LabelGuardedIntCounter<3>, + pub materialize_cache_total_count: LabelGuardedIntCounter<3>, + pub materialize_input_row_count: LabelGuardedIntCounter<3>, +} + +pub struct GroupTopNMetrics { + pub group_top_n_cache_miss_count: LabelGuardedIntCounter<3>, + pub group_top_n_total_query_cache_count: LabelGuardedIntCounter<3>, + pub group_top_n_cached_entry_count: LabelGuardedIntGauge<3>, +} + +pub struct LookupExecutorMetrics { + pub lookup_cache_miss_count: LabelGuardedIntCounter<3>, + pub lookup_total_query_cache_count: LabelGuardedIntCounter<3>, + pub lookup_cached_entry_count: LabelGuardedIntGauge<3>, +} + +pub struct HashAggMetrics { + pub agg_lookup_miss_count: LabelGuardedIntCounter<3>, + pub agg_total_lookup_count: LabelGuardedIntCounter<3>, + pub agg_cached_entry_count: LabelGuardedIntGauge<3>, + pub agg_chunk_lookup_miss_count: LabelGuardedIntCounter<3>, + pub agg_chunk_total_lookup_count: LabelGuardedIntCounter<3>, + pub agg_dirty_groups_count: LabelGuardedIntGauge<3>, + pub agg_dirty_groups_heap_size: LabelGuardedIntGauge<3>, +} + +pub struct AggDistinctDedupMetrics { + pub agg_distinct_cache_miss_count: LabelGuardedIntCounter<3>, + pub agg_distinct_total_cache_count: LabelGuardedIntCounter<3>, + pub agg_distinct_cached_entry_count: LabelGuardedIntGauge<3>, +} + +pub struct TemporalJoinMetrics { + pub temporal_join_cache_miss_count: LabelGuardedIntCounter<3>, + pub temporal_join_total_query_cache_count: LabelGuardedIntCounter<3>, + pub temporal_join_cached_entry_count: LabelGuardedIntGauge<3>, +} + +pub struct BackfillMetrics { + pub backfill_snapshot_read_row_count: LabelGuardedIntCounter<2>, + pub backfill_upstream_output_row_count: LabelGuardedIntCounter<2>, +} + +pub struct CdcBackfillMetrics { + pub cdc_backfill_snapshot_read_row_count: LabelGuardedIntCounter<2>, + pub cdc_backfill_upstream_output_row_count: LabelGuardedIntCounter<2>, +} + +pub struct OverWindowMetrics { + pub over_window_cached_entry_count: LabelGuardedIntGauge<3>, + pub over_window_cache_lookup_count: LabelGuardedIntCounter<3>, + pub over_window_cache_miss_count: LabelGuardedIntCounter<3>, + pub over_window_range_cache_entry_count: LabelGuardedIntGauge<3>, + pub over_window_range_cache_lookup_count: LabelGuardedIntCounter<3>, + pub over_window_range_cache_left_miss_count: LabelGuardedIntCounter<3>, + pub over_window_range_cache_right_miss_count: LabelGuardedIntCounter<3>, } diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index 4f6fc967ad02a..1861313baddc6 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -37,6 +37,7 @@ use risingwave_storage::row_serde::value_serde::{ValueRowSerde, ValueRowSerdeNew use crate::cache::ManagedLruCache; use crate::common::metrics::MetricsInfo; use crate::common::table::state_table::{StateTableInner, StateTableOpConsistencyLevel}; +use crate::executor::monitor::MaterializeMetrics; use crate::executor::prelude::*; /// `MaterializeExecutor` materializes changes in stream into a materialized view on storage. @@ -61,6 +62,8 @@ pub struct MaterializeExecutor { may_have_downstream: bool, depended_subscription_ids: HashSet, + + metrics: MaterializeMetrics, } fn get_op_consistency_level( @@ -129,8 +132,15 @@ impl MaterializeExecutor { ) .await; + let mv_metrics = metrics.new_materialize_metrics( + TableId::new(table_catalog.id), + actor_context.id, + actor_context.fragment_id, + ); + let metrics_info = MetricsInfo::new(metrics, table_catalog.id, actor_context.id, "Materialize"); + Self { input, schema, @@ -147,16 +157,12 @@ impl MaterializeExecutor { version_column_index, may_have_downstream, depended_subscription_ids, + metrics: mv_metrics, } } #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self) { - // for metrics - let table_id_str = self.state_table.table_id().to_string(); - let actor_id_str = self.actor_context.id.to_string(); - let fragment_id_str = self.actor_context.fragment_id.to_string(); - let mv_table_id = TableId::new(self.state_table.table_id()); let data_types = self.schema.data_types(); @@ -176,10 +182,8 @@ impl MaterializeExecutor { yield match msg { Message::Watermark(w) => Message::Watermark(w), Message::Chunk(chunk) => { - self.actor_context - .streaming_metrics - .mview_input_row_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) + self.metrics + .materialize_input_row_count .inc_by(chunk.cardinality() as u64); // This is an optimization that handles conflicts only when a particular materialized view downstream has no MV dependencies. @@ -232,7 +236,12 @@ impl MaterializeExecutor { let fixed_changes = self .materialize_cache - .handle(row_ops, &self.state_table, &self.conflict_behavior) + .handle( + row_ops, + &self.state_table, + &self.conflict_behavior, + &self.metrics, + ) .await?; match generate_output(fixed_changes, data_types.clone())? { @@ -379,6 +388,8 @@ impl MaterializeExecutor { ) .await; + let metrics = StreamingMetrics::unused().new_materialize_metrics(table_id, 1, 2); + Self { input, schema, @@ -395,6 +406,7 @@ impl MaterializeExecutor { version_column_index: None, may_have_downstream: true, depended_subscription_ids: HashSet::new(), + metrics, } } } @@ -538,7 +550,6 @@ impl std::fmt::Debug for MaterializeExecutor { data: ManagedLruCache, CacheValue>, - metrics_info: MetricsInfo, row_serde: BasicSerde, version_column_index: Option, _serde: PhantomData, @@ -557,7 +568,6 @@ impl MaterializeCache { ManagedLruCache::unbounded(watermark_sequence, metrics_info.clone()); Self { data: cache, - metrics_info, row_serde, version_column_index, _serde: PhantomData, @@ -569,13 +579,19 @@ impl MaterializeCache { row_ops: Vec<(Op, Vec, Bytes)>, table: &StateTableInner, conflict_behavior: &ConflictBehavior, + metrics: &MaterializeMetrics, ) -> StreamExecutorResult { let key_set: HashSet> = row_ops .iter() .map(|(_, k, _)| k.as_slice().into()) .collect(); - self.fetch_keys(key_set.iter().map(|v| v.deref()), table, conflict_behavior) - .await?; + self.fetch_keys( + key_set.iter().map(|v| v.deref()), + table, + conflict_behavior, + metrics, + ) + .await?; let mut fixed_changes = MaterializeBuffer::new(); let row_serde = self.row_serde.clone(); let version_column_index = self.version_column_index; @@ -720,20 +736,14 @@ impl MaterializeCache { keys: impl Iterator, table: &StateTableInner, conflict_behavior: &ConflictBehavior, + metrics: &MaterializeMetrics, ) -> StreamExecutorResult<()> { let mut futures = vec![]; for key in keys { - self.metrics_info - .metrics - .materialize_cache_total_count - .with_label_values(&[&self.metrics_info.table_id, &self.metrics_info.actor_id]) - .inc(); + metrics.materialize_cache_total_count.inc(); + if self.data.contains(key) { - self.metrics_info - .metrics - .materialize_cache_hit_count - .with_label_values(&[&self.metrics_info.table_id, &self.metrics_info.actor_id]) - .inc(); + metrics.materialize_cache_hit_count.inc(); continue; } futures.push(async { diff --git a/src/stream/src/executor/over_window/general.rs b/src/stream/src/executor/over_window/general.rs index 969ce96fa334f..815e3b6698a51 100644 --- a/src/stream/src/executor/over_window/general.rs +++ b/src/stream/src/executor/over_window/general.rs @@ -37,6 +37,7 @@ use super::over_partition::{ }; use crate::cache::ManagedLruCache; use crate::common::metrics::MetricsInfo; +use crate::executor::monitor::OverWindowMetrics; use crate::executor::over_window::over_partition::AffectedRange; use crate::executor::prelude::*; @@ -65,7 +66,6 @@ struct ExecutorInner { state_table: StateTable, watermark_sequence: AtomicU64Ref, - metrics: Arc, /// The maximum size of the chunk produced by executor at a time. chunk_size: usize, @@ -183,7 +183,6 @@ impl OverWindowExecutor { state_key_to_table_sub_pk_proj, state_table: args.state_table, watermark_sequence: args.watermark_epoch, - metrics: args.metrics, chunk_size: args.chunk_size, cache_policy, }, @@ -249,6 +248,7 @@ impl OverWindowExecutor { this: &'a mut ExecutorInner, vars: &'a mut ExecutionVars, chunk: StreamChunk, + metrics: &'a OverWindowMetrics, ) { // partition key => changes happened in the partition. let mut deltas: BTreeMap, PartitionDelta> = BTreeMap::new(); @@ -307,11 +307,6 @@ impl OverWindowExecutor { BTreeMap::new(); let mut chunk_builder = StreamChunkBuilder::new(this.chunk_size, this.schema.data_types()); - // Prepare things needed by metrics. - let actor_id = this.actor_ctx.id.to_string(); - let fragment_id = this.actor_ctx.fragment_id.to_string(); - let table_id = this.state_table.table_id().to_string(); - // Build final changes partition by partition. for (part_key, delta) in deltas { vars.stats.cache_lookup += 1; @@ -382,22 +377,17 @@ impl OverWindowExecutor { let cache_len = partition.cache_real_len(); let stats = partition.summarize(); - let metrics = this.actor_ctx.streaming_metrics.clone(); metrics .over_window_range_cache_entry_count - .with_label_values(&[&table_id, &actor_id, &fragment_id]) .set(cache_len as i64); metrics .over_window_range_cache_lookup_count - .with_label_values(&[&table_id, &actor_id, &fragment_id]) .inc_by(stats.lookup_count); metrics .over_window_range_cache_left_miss_count - .with_label_values(&[&table_id, &actor_id, &fragment_id]) .inc_by(stats.left_miss_count); metrics .over_window_range_cache_right_miss_count - .with_label_values(&[&table_id, &actor_id, &fragment_id]) .inc_by(stats.right_miss_count); // Update recently accessed range for later shrinking cache. @@ -597,6 +587,12 @@ impl OverWindowExecutor { "OverWindow", ); + let metrics = metrics_info.metrics.new_over_window_metrics( + this.state_table.table_id(), + this.actor_ctx.id, + this.actor_ctx.fragment_id, + ); + let mut vars = ExecutionVars { cached_partitions: ManagedLruCache::unbounded( this.watermark_sequence.clone(), @@ -624,7 +620,7 @@ impl OverWindowExecutor { } Message::Chunk(chunk) => { #[for_await] - for chunk in Self::apply_chunk(&mut this, &mut vars, chunk) { + for chunk in Self::apply_chunk(&mut this, &mut vars, chunk, &metrics) { yield Message::Chunk(chunk?); } this.state_table.try_flush().await?; @@ -633,24 +629,15 @@ impl OverWindowExecutor { this.state_table.commit(barrier.epoch).await?; vars.cached_partitions.evict(); - { - // update metrics - let actor_id_str = this.actor_ctx.id.to_string(); - let fragment_id_str = this.actor_ctx.fragment_id.to_string(); - let table_id_str = this.state_table.table_id().to_string(); - this.metrics - .over_window_cached_entry_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) - .set(vars.cached_partitions.len() as _); - this.metrics - .over_window_cache_lookup_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) - .inc_by(std::mem::take(&mut vars.stats.cache_lookup)); - this.metrics - .over_window_cache_miss_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) - .inc_by(std::mem::take(&mut vars.stats.cache_miss)); - } + metrics + .over_window_cached_entry_count + .set(vars.cached_partitions.len() as _); + metrics + .over_window_cache_lookup_count + .inc_by(std::mem::take(&mut vars.stats.cache_lookup)); + metrics + .over_window_cache_miss_count + .inc_by(std::mem::take(&mut vars.stats.cache_miss)); if let Some(vnode_bitmap) = barrier.as_update_vnode_bitmap(this.actor_ctx.id) { let (_, cache_may_stale) = diff --git a/src/stream/src/executor/receiver.rs b/src/stream/src/executor/receiver.rs index 1493e3ce17cf9..effe773d54594 100644 --- a/src/stream/src/executor/receiver.rs +++ b/src/stream/src/executor/receiver.rs @@ -19,7 +19,6 @@ use tokio::time::Instant; use super::exchange::input::BoxedInput; use crate::executor::exchange::input::new_input; use crate::executor::prelude::*; -use crate::executor::utils::ActorInputMetrics; use crate::task::{FragmentId, SharedContext}; /// `ReceiverExecutor` is used along with a channel. After creating a mpsc channel, @@ -93,8 +92,7 @@ impl Execute for ReceiverExecutor { fn execute(mut self: Box) -> BoxedMessageStream { let actor_id = self.actor_context.id; - let mut metrics = ActorInputMetrics::new( - &self.metrics, + let mut metrics = self.metrics.new_actor_input_metrics( actor_id, self.fragment_id, self.upstream_fragment_id, @@ -169,8 +167,7 @@ impl Execute for ReceiverExecutor { self.input = new_upstream; self.upstream_fragment_id = new_upstream_fragment_id; - metrics = ActorInputMetrics::new( - &self.metrics, + metrics = self.metrics.new_actor_input_metrics( actor_id, self.fragment_id, self.upstream_fragment_id, diff --git a/src/stream/src/executor/simple_agg.rs b/src/stream/src/executor/simple_agg.rs index c6c2701b276f6..3ed0c2533e45d 100644 --- a/src/stream/src/executor/simple_agg.rs +++ b/src/stream/src/executor/simple_agg.rs @@ -236,7 +236,7 @@ impl SimpleAggExecutor { &this.agg_calls, this.watermark_epoch.clone(), &this.distinct_dedup_tables, - this.actor_ctx.clone(), + &this.actor_ctx, ); yield Message::Barrier(barrier); diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs index e12590527e3a8..661b1efbcc0b1 100644 --- a/src/stream/src/executor/sink.rs +++ b/src/stream/src/executor/sink.rs @@ -21,11 +21,11 @@ use itertools::Itertools; use risingwave_common::array::stream_chunk::StreamChunkMut; use risingwave_common::array::Op; use risingwave_common::catalog::{ColumnCatalog, Field}; -use risingwave_common::metrics::GLOBAL_ERROR_METRICS; +use risingwave_common::metrics::{LabelGuardedIntGauge, GLOBAL_ERROR_METRICS}; use risingwave_common_estimate_size::collections::EstimatedVec; use risingwave_common_estimate_size::EstimateSize; use risingwave_connector::dispatch_sink; -use risingwave_connector::sink::catalog::{SinkId, SinkType}; +use risingwave_connector::sink::catalog::SinkType; use risingwave_connector::sink::log_store::{ LogReader, LogReaderExt, LogStoreFactory, LogWriter, LogWriterExt, }; @@ -124,9 +124,15 @@ impl SinkExecutor { fn execute_inner(self) -> BoxedMessageStream { let sink_id = self.sink_param.sink_id; let actor_id = self.actor_context.id; + let fragment_id = self.actor_context.fragment_id; let executor_id = self.sink_writer_param.executor_id; let stream_key = self.info.pk_indices.clone(); + let metrics = self.actor_context.streaming_metrics.new_sink_exec_metrics( + sink_id, + actor_id, + fragment_id, + ); let stream_key_sink_pk_mismatch = { stream_key @@ -136,19 +142,9 @@ impl SinkExecutor { let input = self.input.execute(); - let input_row_count = self - .actor_context - .streaming_metrics - .sink_input_row_count - .with_guarded_label_values(&[ - &sink_id.to_string(), - &actor_id.to_string(), - &self.actor_context.fragment_id.to_string(), - ]); - let input = input.inspect_ok(move |msg| { if let Message::Chunk(c) = msg { - input_row_count.inc_by(c.capacity() as u64); + metrics.sink_input_row_count.inc_by(c.capacity() as u64); } }); @@ -190,7 +186,6 @@ impl SinkExecutor { && !self.sink_param.downstream_pk.is_empty(); let processed_input = Self::process_msg( input, - self.sink_param.sink_id, self.sink_param.sink_type, stream_key, need_advance_delete, @@ -198,7 +193,7 @@ impl SinkExecutor { self.chunk_size, self.input_data_types, self.sink_param.downstream_pk.clone(), - self.actor_context.clone(), + metrics.sink_chunk_buffer_size, ); if self.sink.is_sink_into_table() { @@ -297,7 +292,6 @@ impl SinkExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] async fn process_msg( input: impl MessageStream, - sink_id: SinkId, sink_type: SinkType, stream_key: PkIndices, need_advance_delete: bool, @@ -305,7 +299,7 @@ impl SinkExecutor { chunk_size: usize, input_data_types: Vec, down_stream_pk: Vec, - actor_context: ActorContextRef, + sink_chunk_buffer_size_metrics: LabelGuardedIntGauge<3>, ) { // need to buffer chunks during one barrier if need_advance_delete || re_construct_with_sink_pk { @@ -317,15 +311,8 @@ impl SinkExecutor { Message::Watermark(w) => watermark = Some(w), Message::Chunk(c) => { chunk_buffer.push(c); - actor_context - .streaming_metrics - .sink_chunk_buffer_size - .with_guarded_label_values(&[ - &sink_id.to_string(), - &actor_context.id.to_string(), - &actor_context.fragment_id.to_string(), - ]) - .set(chunk_buffer.estimated_size() as i64); + + sink_chunk_buffer_size_metrics.set(chunk_buffer.estimated_size() as i64); } Message::Barrier(barrier) => { let chunks = mem::take(&mut chunk_buffer).into_inner(); diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 5e597b830dd92..7dd999d11fb1a 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -386,6 +386,17 @@ impl FsSourceExecutor { self.system_params.load().barrier_interval_ms() as u128 * WAIT_BARRIER_MULTIPLE_TIMES; let mut last_barrier_time = Instant::now(); let mut self_paused = false; + + let source_output_row_count = self + .metrics + .source_output_row_count + .with_guarded_label_values(&[ + self.stream_source_core.source_id.to_string().as_ref(), + self.stream_source_core.source_name.as_ref(), + self.actor_ctx.id.to_string().as_str(), + self.actor_ctx.fragment_id.to_string().as_str(), + ]); + while let Some(msg) = stream.next().await { match msg? { // This branch will be preferred. @@ -474,15 +485,7 @@ impl FsSourceExecutor { .extend(state); } - self.metrics - .source_output_row_count - .with_label_values(&[ - self.stream_source_core.source_id.to_string().as_ref(), - self.stream_source_core.source_name.as_ref(), - self.actor_ctx.id.to_string().as_str(), - self.actor_ctx.fragment_id.to_string().as_str(), - ]) - .inc_by(chunk.cardinality() as u64); + source_output_row_count.inc_by(chunk.cardinality() as u64); let chunk = prune_additional_cols(&chunk, split_idx, offset_idx, &source_desc.columns); diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index 29ef22b60fcb2..7135a3fc3bf8e 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -21,9 +21,8 @@ use anyhow::anyhow; use either::Either; use futures::stream::{select_with_strategy, PollNext}; use itertools::Itertools; -use prometheus::IntCounter; use risingwave_common::buffer::BitmapBuilder; -use risingwave_common::metrics::GLOBAL_ERROR_METRICS; +use risingwave_common::metrics::{LabelGuardedIntCounter, GLOBAL_ERROR_METRICS}; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::types::JsonbVal; @@ -127,7 +126,7 @@ pub struct SourceBackfillExecutorInner { /// Metrics for monitor. metrics: Arc, - source_split_change_count: IntCounter, + source_split_change_count: LabelGuardedIntCounter<4>, // /// Receiver of barrier channel. // barrier_receiver: Option>, @@ -176,12 +175,14 @@ impl SourceBackfillExecutorInner { backfill_state_store: BackfillStateTableHandler, rate_limit_rps: Option, ) -> Self { - let source_split_change_count = metrics.source_split_change_count.with_label_values(&[ - &stream_source_core.source_id.to_string(), - &stream_source_core.source_name, - &actor_ctx.id.to_string(), - &actor_ctx.fragment_id.to_string(), - ]); + let source_split_change_count = metrics + .source_split_change_count + .with_guarded_label_values(&[ + &stream_source_core.source_id.to_string(), + &stream_source_core.source_name, + &actor_ctx.id.to_string(), + &actor_ctx.fragment_id.to_string(), + ]); Self { actor_ctx, info, diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 4bfd1f74a33fc..4236a968d92d8 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -20,7 +20,7 @@ use either::Either; use futures::TryStreamExt; use itertools::Itertools; use risingwave_common::array::ArrayRef; -use risingwave_common::metrics::GLOBAL_ERROR_METRICS; +use risingwave_common::metrics::{LabelGuardedIntCounter, GLOBAL_ERROR_METRICS}; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::util::epoch::{Epoch, EpochPair}; @@ -180,17 +180,9 @@ impl SourceExecutor { stream: &mut StreamReaderWithPause, split_assignment: &HashMap>, should_trim_state: bool, + source_split_change_count_metrics: &LabelGuardedIntCounter<4>, ) -> StreamExecutorResult<()> { - self.metrics - .source_split_change_count - .with_label_values( - &self - .get_metric_labels() - .iter() - .map(AsRef::as_ref) - .collect::>(), - ) - .inc(); + source_split_change_count_metrics.inc(); if let Some(target_splits) = split_assignment.get(&self.actor_ctx.id).cloned() { if self .update_state_if_changed(target_splits, should_trim_state) @@ -495,6 +487,16 @@ impl SourceExecutor { let mut last_barrier_time = Instant::now(); let mut self_paused = false; + let source_output_row_count = self + .metrics + .source_output_row_count + .with_guarded_label_values(&self.get_metric_labels().each_ref().map(AsRef::as_ref)); + + let source_split_change_count = self + .metrics + .source_split_change_count + .with_guarded_label_values(&self.get_metric_labels().each_ref().map(AsRef::as_ref)); + while let Some(msg) = stream.next().await { let Ok(msg) = msg else { tokio::time::sleep(Duration::from_millis(1000)).await; @@ -540,6 +542,7 @@ impl SourceExecutor { &mut stream, actor_splits, true, + &source_split_change_count, ) .await?; } @@ -550,6 +553,7 @@ impl SourceExecutor { &mut stream, actor_splits, false, + &source_split_change_count, ) .await?; } @@ -641,16 +645,7 @@ impl SourceExecutor { .extend(state); } - self.metrics - .source_output_row_count - .with_label_values( - &self - .get_metric_labels() - .iter() - .map(AsRef::as_ref) - .collect::>(), - ) - .inc_by(chunk.cardinality() as u64); + source_output_row_count.inc_by(chunk.cardinality() as u64); let chunk = prune_additional_cols(&chunk, split_idx, offset_idx, &source_desc.columns); yield Message::Chunk(chunk); diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs index a5693a763eb31..077ea17ca74d9 100644 --- a/src/stream/src/executor/temporal_join.rs +++ b/src/stream/src/executor/temporal_join.rs @@ -35,6 +35,7 @@ use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::TableIter; use super::join::{JoinType, JoinTypePrimitive}; +use super::monitor::TemporalJoinMetrics; use crate::cache::{cache_may_stale, ManagedLruCache}; use crate::common::metrics::MetricsInfo; use crate::executor::join::builder::JoinStreamChunkBuilder; @@ -59,6 +60,7 @@ pub struct TemporalJoinExecutor< output_indices: Vec, chunk_size: usize, memo_table: Option>, + metrics: TemporalJoinMetrics, } #[derive(Default)] @@ -106,7 +108,6 @@ struct TemporalSide { table_stream_key_indices: Vec, table_output_indices: Vec, cache: ManagedLruCache>, - ctx: ActorContextRef, join_key_data_types: Vec, } @@ -117,25 +118,14 @@ impl TemporalSide { &mut self, keys: impl Iterator, epoch: HummockEpoch, + metrics: &TemporalJoinMetrics, ) -> StreamExecutorResult<()> { - let table_id_str = self.source.table_id().to_string(); - let actor_id_str = self.ctx.id.to_string(); - let fragment_id_str = self.ctx.id.to_string(); - let mut futs = Vec::with_capacity(keys.size_hint().1.unwrap_or(0)); for key in keys { - self.ctx - .streaming_metrics - .temporal_join_total_query_cache_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) - .inc(); + metrics.temporal_join_total_query_cache_count.inc(); if self.cache.get(key).is_none() { - self.ctx - .streaming_metrics - .temporal_join_cache_miss_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) - .inc(); + metrics.temporal_join_cache_miss_count.inc(); futs.push(async { let pk_prefix = key.deserialize(&self.join_key_data_types)?; @@ -318,6 +308,7 @@ mod phase1 { use super::{StreamExecutorError, TemporalSide}; use crate::common::table::state_table::StateTable; + use crate::executor::monitor::TemporalJoinMetrics; pub(super) trait Phase1Evaluation { /// Called when a matched row is found. @@ -439,6 +430,7 @@ mod phase1 { memo_table: &'a mut Option>, null_matched: &'a K::Bitmap, chunk: StreamChunk, + metrics: &'a TemporalJoinMetrics, ) { let mut builder = StreamChunkBuilder::new(chunk_size, full_schema); let keys = K::build_many(left_join_keys, chunk.data_chunk()); @@ -463,7 +455,7 @@ mod phase1 { } }); right_table - .fetch_or_promote_keys(to_fetch_keys, epoch) + .fetch_or_promote_keys(to_fetch_keys, epoch, metrics) .await?; for (r, key) in chunk.rows_with_holes().zip_eq_debug(keys.into_iter()) { @@ -618,6 +610,8 @@ impl alloc, ); + let metrics = metrics.new_temporal_join_metrics(table.table_id(), ctx.id, ctx.fragment_id); + Self { ctx: ctx.clone(), info, @@ -628,7 +622,6 @@ impl table_stream_key_indices, table_output_indices, cache, - ctx, join_key_data_types, }, left_join_keys, @@ -638,6 +631,7 @@ impl output_indices, chunk_size, memo_table, + metrics, } } @@ -677,9 +671,6 @@ impl let mut prev_epoch = None; - let table_id_str = self.right_table.source.table_id().to_string(); - let actor_id_str = self.ctx.id.to_string(); - let fragment_id_str = self.ctx.fragment_id.to_string(); let full_schema: Vec<_> = self .left .schema() @@ -693,10 +684,8 @@ impl #[for_await] for msg in align_input(self.left, self.right) { self.right_table.cache.evict(); - self.ctx - .streaming_metrics + self.metrics .temporal_join_cached_entry_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(self.right_table.cache.len() as i64); match msg? { InternalMessage::WaterMark(watermark) => { @@ -720,6 +709,7 @@ impl &mut self.memo_table, &null_matched, chunk, + &self.metrics, ); #[for_await] for chunk in st1 { @@ -752,6 +742,7 @@ impl &mut self.memo_table, &null_matched, chunk, + &self.metrics, ); let mut matched_count = 0usize; #[for_await] @@ -803,6 +794,7 @@ impl &mut self.memo_table, &null_matched, chunk, + &self.metrics, ); #[for_await] for chunk in st1 { diff --git a/src/stream/src/executor/top_n/group_top_n.rs b/src/stream/src/executor/top_n/group_top_n.rs index 7f1c21fd14610..9457e2f7729c6 100644 --- a/src/stream/src/executor/top_n/group_top_n.rs +++ b/src/stream/src/executor/top_n/group_top_n.rs @@ -27,6 +27,7 @@ use super::utils::*; use super::{ManagedTopNState, TopNCache}; use crate::cache::ManagedLruCache; use crate::common::metrics::MetricsInfo; +use crate::executor::monitor::GroupTopNMetrics; use crate::executor::prelude::*; pub type GroupTopNExecutor = @@ -45,20 +46,17 @@ impl GroupTopNExecutor, watermark_epoch: AtomicU64Ref, ) -> StreamResult { - Ok(TopNExecutorWrapper { - input, - ctx: ctx.clone(), - inner: InnerGroupTopNExecutor::new( - schema, - storage_key, - offset_and_limit, - order_by, - group_by, - state_table, - watermark_epoch, - ctx, - )?, - }) + let inner = InnerGroupTopNExecutor::new( + schema, + storage_key, + offset_and_limit, + order_by, + group_by, + state_table, + watermark_epoch, + &ctx, + )?; + Ok(TopNExecutorWrapper { input, ctx, inner }) } } @@ -85,7 +83,7 @@ pub struct InnerGroupTopNExecutor InnerGroupTopNExecutor { @@ -98,7 +96,7 @@ impl InnerGroupTopNExecutor, state_table: StateTable, watermark_epoch: AtomicU64Ref, - ctx: ActorContextRef, + ctx: &ActorContext, ) -> StreamResult { let metrics_info = MetricsInfo::new( ctx.streaming_metrics.clone(), @@ -106,6 +104,11 @@ impl InnerGroupTopNExecutor::new(state_table, cache_key_serde.clone()); @@ -119,7 +122,7 @@ impl InnerGroupTopNExecutor state_table: StateTable, watermark_epoch: AtomicU64Ref, ) -> StreamResult { - Ok(TopNExecutorWrapper { - input, - ctx: ctx.clone(), - inner: InnerAppendOnlyGroupTopNExecutor::new( - schema, - storage_key, - offset_and_limit, - order_by, - group_by, - state_table, - watermark_epoch, - ctx, - )?, - }) + let inner = InnerAppendOnlyGroupTopNExecutor::new( + schema, + storage_key, + offset_and_limit, + order_by, + group_by, + state_table, + watermark_epoch, + &ctx, + )?; + Ok(TopNExecutorWrapper { input, ctx, inner }) } } @@ -88,7 +86,7 @@ pub struct InnerAppendOnlyGroupTopNExecutor @@ -103,13 +101,18 @@ impl group_by: Vec, state_table: StateTable, watermark_epoch: AtomicU64Ref, - ctx: ActorContextRef, + ctx: &ActorContext, ) -> StreamResult { let metrics_info = MetricsInfo::new( ctx.streaming_metrics.clone(), state_table.table_id(), ctx.id, - "GroupTopN", + "AppendOnlyGroupTopN", + ); + let metrics = ctx.streaming_metrics.new_append_only_group_top_n_metrics( + state_table.table_id(), + ctx.id, + ctx.fragment_id, ); let cache_key_serde = create_cache_key_serde(&storage_key, &schema, &order_by, &group_by); @@ -124,7 +127,7 @@ impl group_by, caches: GroupTopNCache::new(watermark_epoch, metrics_info), cache_key_serde, - ctx, + metrics, }) } } @@ -141,9 +144,6 @@ where let data_types = self.schema.data_types(); let row_deserializer = RowDeserializer::new(data_types.clone()); - let table_id_str = self.managed_state.table().table_id().to_string(); - let actor_id_str = self.ctx.id.to_string(); - let fragment_id_str = self.ctx.fragment_id.to_string(); for (r, group_cache_key) in chunk.rows_with_holes().zip_eq_debug(keys.iter()) { let Some((op, row_ref)) = r else { continue; @@ -153,19 +153,11 @@ where let cache_key = serialize_pk_to_cache_key(pk_row, &self.cache_key_serde); let group_key = row_ref.project(&self.group_by); - self.ctx - .streaming_metrics - .group_top_n_appendonly_total_query_cache_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) - .inc(); + self.metrics.group_top_n_total_query_cache_count.inc(); // If 'self.caches' does not already have a cache for the current group, create a new // cache for it and insert it into `self.caches` if !self.caches.contains(group_cache_key) { - self.ctx - .streaming_metrics - .group_top_n_appendonly_cache_miss_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) - .inc(); + self.metrics.group_top_n_cache_miss_count.inc(); let mut topn_cache = TopNCache::new(self.offset, self.limit, data_types.clone()); self.managed_state .init_topn_cache(Some(group_key), &mut topn_cache) @@ -184,10 +176,8 @@ where &row_deserializer, )?; } - self.ctx - .streaming_metrics - .group_top_n_appendonly_cached_entry_count - .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) + self.metrics + .group_top_n_cached_entry_count .set(self.caches.len() as i64); generate_output(res_rows, res_ops, &self.schema) } diff --git a/src/stream/src/executor/utils.rs b/src/stream/src/executor/utils.rs index 565866e0d9e95..37ed7803a44c7 100644 --- a/src/stream/src/executor/utils.rs +++ b/src/stream/src/executor/utils.rs @@ -12,10 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::metrics::LabelGuardedIntCounter; - use crate::executor::prelude::*; -use crate::task::FragmentId; #[derive(Default)] pub struct DummyExecutor; @@ -25,35 +22,3 @@ impl Execute for DummyExecutor { futures::stream::pending().boxed() } } - -pub(crate) struct ActorInputMetrics { - pub(crate) actor_in_record_cnt: LabelGuardedIntCounter<3>, - pub(crate) actor_input_buffer_blocking_duration_ns: LabelGuardedIntCounter<3>, -} - -impl ActorInputMetrics { - pub(crate) fn new( - metrics: &StreamingMetrics, - actor_id: ActorId, - fragment_id: FragmentId, - upstream_fragment_id: FragmentId, - ) -> Self { - let actor_id_str = actor_id.to_string(); - let fragment_id_str = fragment_id.to_string(); - let upstream_fragment_id_str = upstream_fragment_id.to_string(); - Self { - actor_in_record_cnt: metrics.actor_in_record_cnt.with_guarded_label_values(&[ - &actor_id_str, - &fragment_id_str, - &upstream_fragment_id_str, - ]), - actor_input_buffer_blocking_duration_ns: metrics - .actor_input_buffer_blocking_duration_ns - .with_guarded_label_values(&[ - &actor_id_str, - &fragment_id_str, - &upstream_fragment_id_str, - ]), - } - } -} diff --git a/src/stream/src/executor/wrapper/trace.rs b/src/stream/src/executor/wrapper/trace.rs index c95809f534728..4c1268cc8698b 100644 --- a/src/stream/src/executor/wrapper/trace.rs +++ b/src/stream/src/executor/wrapper/trace.rs @@ -33,6 +33,11 @@ pub async fn trace( let actor_id_str = actor_ctx.id.to_string(); let fragment_id_str = actor_ctx.fragment_id.to_string(); + let executor_row_count = actor_ctx + .streaming_metrics + .executor_row_count + .with_guarded_label_values(&[&actor_id_str, &fragment_id_str, &info.identity]); + let new_span = || { tracing::info_span!( "executor", @@ -50,11 +55,7 @@ pub async fn trace( match &message { Message::Chunk(chunk) => { if enable_executor_row_count { - actor_ctx - .streaming_metrics - .executor_row_count - .with_label_values(&[&actor_id_str, &fragment_id_str, &info.identity]) - .inc_by(chunk.cardinality() as u64); + executor_row_count.inc_by(chunk.cardinality() as u64); } tracing::debug!( target: "events::stream::message::chunk", diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 88f9d3e27d2ea..f9fa061993a79 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -658,54 +658,43 @@ impl LocalBarrierWorker { if self.actor_manager.streaming_metrics.level >= MetricLevel::Debug { tracing::info!("Tokio metrics are enabled because metrics_level >= Debug"); - let actor_id_str = actor_id.to_string(); - let metrics = self.actor_manager.streaming_metrics.clone(); + let streaming_metrics = self.actor_manager.streaming_metrics.clone(); let actor_monitor_task = self.actor_manager.runtime.spawn(async move { + let metrics = streaming_metrics.new_actor_metrics(actor_id); loop { let task_metrics = monitor.cumulative(); metrics .actor_execution_time - .with_label_values(&[&actor_id_str]) .set(task_metrics.total_poll_duration.as_secs_f64()); metrics .actor_fast_poll_duration - .with_label_values(&[&actor_id_str]) .set(task_metrics.total_fast_poll_duration.as_secs_f64()); metrics .actor_fast_poll_cnt - .with_label_values(&[&actor_id_str]) .set(task_metrics.total_fast_poll_count as i64); metrics .actor_slow_poll_duration - .with_label_values(&[&actor_id_str]) .set(task_metrics.total_slow_poll_duration.as_secs_f64()); metrics .actor_slow_poll_cnt - .with_label_values(&[&actor_id_str]) .set(task_metrics.total_slow_poll_count as i64); metrics .actor_poll_duration - .with_label_values(&[&actor_id_str]) .set(task_metrics.total_poll_duration.as_secs_f64()); metrics .actor_poll_cnt - .with_label_values(&[&actor_id_str]) .set(task_metrics.total_poll_count as i64); metrics .actor_idle_duration - .with_label_values(&[&actor_id_str]) .set(task_metrics.total_idle_duration.as_secs_f64()); metrics .actor_idle_cnt - .with_label_values(&[&actor_id_str]) .set(task_metrics.total_idled_count as i64); metrics .actor_scheduled_duration - .with_label_values(&[&actor_id_str]) .set(task_metrics.total_scheduled_duration.as_secs_f64()); metrics .actor_scheduled_cnt - .with_label_values(&[&actor_id_str]) .set(task_metrics.total_scheduled_count as i64); tokio::time::sleep(Duration::from_secs(1)).await; } From a493616e67701861ad3969dc5fc09d4c8bd665dc Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 29 May 2024 02:20:15 +0800 Subject: [PATCH 11/13] chore(ci): increase timeout for slow e2e tests (#16980) --- ci/workflows/main-cron.yml | 2 +- ci/workflows/pull-request.yml | 2 +- e2e_test_slow/udf/always_retry_python.slt | 11 ++++++----- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index d9ddb954b2f17..e097d2d587994 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -106,7 +106,7 @@ steps: config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 4 + timeout_in_minutes: 8 retry: *auto-retry - label: "meta backup test (release)" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 2f91e23289da0..9ced38891a75b 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -134,7 +134,7 @@ steps: config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 6 + timeout_in_minutes: 8 retry: *auto-retry - label: "end-to-end test (parallel)" diff --git a/e2e_test_slow/udf/always_retry_python.slt b/e2e_test_slow/udf/always_retry_python.slt index 1889bd136bb97..78bf926c32986 100644 --- a/e2e_test_slow/udf/always_retry_python.slt +++ b/e2e_test_slow/udf/always_retry_python.slt @@ -15,7 +15,7 @@ statement ok CREATE TABLE t (v1 int); statement ok -INSERT INTO t select 0 from generate_series(1, 30); +INSERT INTO t select 0 from generate_series(1, 60); statement ok flush; @@ -50,10 +50,11 @@ SELECT count(*) FROM mv_always_retry where s1 is NULL; ---- 0 -query B -SELECT count(*) > 0 FROM mv_no_retry where s1 is NULL; ----- -t +# FIXME(kwannoel): Somehow this is flaky.. +# query B +# SELECT count(*) > 0 FROM mv_no_retry where s1 is NULL; +# ---- +# t statement ok SET STREAMING_RATE_LIMIT TO DEFAULT; From 12f5c0d26594eb20e815972f334ff6478eec6e76 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Wed, 29 May 2024 05:12:32 +0800 Subject: [PATCH 12/13] refactor(postgres-cdc): refactor postgres_row_to_owned_row (#16714) --- e2e_test/source/cdc/cdc.check_new_rows.slt | 141 +++++++ e2e_test/source/cdc/cdc.share_stream.slt | 10 +- e2e_test/source/cdc/postgres_cdc.sql | 15 +- e2e_test/source/cdc/postgres_cdc_insert.sql | 7 + src/common/src/types/datetime.rs | 44 ++- src/common/src/types/from_sql.rs | 66 ++++ src/common/src/types/jsonb.rs | 40 +- src/common/src/types/mod.rs | 1 + src/common/src/types/timestamptz.rs | 16 +- src/common/src/types/to_sql.rs | 57 +-- src/connector/src/parser/postgres.rs | 364 ++---------------- src/connector/src/parser/scalar_adapter.rs | 265 +++++++++---- .../src/source/cdc/external/postgres.rs | 4 +- 13 files changed, 576 insertions(+), 454 deletions(-) create mode 100644 src/common/src/types/from_sql.rs diff --git a/e2e_test/source/cdc/cdc.check_new_rows.slt b/e2e_test/source/cdc/cdc.check_new_rows.slt index 25189eff09e6e..cac4896f3fa58 100644 --- a/e2e_test/source/cdc/cdc.check_new_rows.slt +++ b/e2e_test/source/cdc/cdc.check_new_rows.slt @@ -128,6 +128,8 @@ select id, my_int from list_with_null_shared order by id; 2 {NULL,3,4} 3 {NULL,-3,-4} 4 {-4,-5,-6} +5 NULL +6 NULL # my_num: varchar[] query II @@ -137,6 +139,8 @@ select id, my_num from list_with_null_shared order by id; 2 {2.2,0,NULL} 3 NULL 4 {NULL,-99999999999999999.9999} +5 NULL +6 NULL # my_num1: numeric[] query II @@ -146,6 +150,8 @@ select id, my_num_1 from list_with_null_shared order by id; 2 {2.2,0,NULL} 3 NULL 4 {NULL,-99999999999999999.9999} +5 NULL +6 NULL # my_num2: rw_int256[] query II @@ -155,6 +161,8 @@ select id, my_num_2 from list_with_null_shared order by id; 2 NULL 3 NULL 4 NULL +5 NULL +6 NULL # Due to the bug in Debezium, if a enum list contains `NULL`, the list will be converted to `NULL` query II @@ -164,6 +172,8 @@ select id, my_mood from list_with_null_shared order by id; 2 {happy,ok,sad} 3 NULL 4 NULL +5 NULL +6 NULL query II select id, my_uuid from list_with_null_shared order by id; @@ -172,6 +182,8 @@ select id, my_uuid from list_with_null_shared order by id; 2 {2de296df-eda7-4202-a81f-1036100ef4f6,2977afbc-0b12-459c-a36f-f623fc9e9840} 3 {NULL,471acecf-a4b4-4ed3-a211-7fb2291f159f,9bc35adf-fb11-4130-944c-e7eadb96b829} 4 {b2e4636d-fa03-4ad4-bf16-029a79dca3e2} +5 NULL +6 NULL query II select id, my_bytea from list_with_null_shared order by id; @@ -180,3 +192,132 @@ select id, my_bytea from list_with_null_shared order by id; 2 {"\\x00","\\x01","\\x02"} 3 {NULL,"\\x99","\\xaa"} 4 {"\\x88","\\x99","\\xaa"} +5 NULL +6 NULL + +query TTTTTTT +SELECT c_boolean, c_smallint, c_integer, c_bigint, c_decimal, c_real, c_double_precision +FROM postgres_all_types +ORDER BY c_boolean, c_bigint, c_date; +---- +f -32767 -2147483647 -9223372036854775807 -10.0 -10000 -10000 +f 0 0 0 0 0 0 +f NULL NULL 1 NULL NULL NULL +f 1 123 1234567890 123.45 123.45 123.456 +t -32767 -2147483647 -9223372036854775807 -10.0 -10000 -10000 +t 0 0 0 0 0 0 +t NULL NULL 1 NULL NULL NULL +t 1 123 1234567890 123.45 123.45 123.456 + +query TTTTTTT +SELECT c_varchar, c_bytea +FROM postgres_all_types +ORDER BY c_boolean, c_bigint, c_date; +---- +d \x00 +(empty) \x00 +NULL NULL +example \xdeadbeef +d \x00 +(empty) \x00 +NULL NULL +example \xdeadbeef + +query TTTTTTT +SELECT c_date, c_time, c_timestamp, c_timestamptz, c_interval +FROM postgres_all_types +ORDER BY c_boolean, c_bigint, c_date; +---- +0001-01-01 00:00:00 2001-01-01 00:00:00 2001-01-01 08:00:00+00:00 00:00:00 +0001-01-01 00:00:00 2001-01-01 00:00:00 2001-01-01 08:00:00+00:00 00:00:00 +0024-05-19 NULL NULL NULL NULL +0024-01-01 12:34:56 2024-05-19 12:34:56 2024-05-19 12:34:56+00:00 1 day +0001-01-01 00:00:00 2001-01-01 00:00:00 2001-01-01 08:00:00+00:00 00:00:00 +0001-01-01 00:00:00 2001-01-01 00:00:00 2001-01-01 08:00:00+00:00 00:00:00 +0024-05-19 NULL NULL NULL NULL +0024-01-01 12:34:56 2024-05-19 12:34:56 2024-05-19 12:34:56+00:00 1 day + +query TTTTTTT +SELECT c_jsonb, c_uuid, c_enum +FROM postgres_all_types +ORDER BY c_boolean, c_bigint, c_date; +---- +{} bb488f9b-330d-4012-b849-12adeb49e57e happy +{} NULL sad +NULL NULL NULL +{"key": "value"} 123e4567-e89b-12d3-a456-426614174000 happy +{} bb488f9b-330d-4012-b849-12adeb49e57e happy +{} NULL sad +NULL NULL NULL +{"key": "value"} 123e4567-e89b-12d3-a456-426614174000 happy + +query TTTTTTT +SELECT c_boolean_array, c_smallint_array, c_integer_array, c_bigint_array +FROM postgres_all_types +ORDER BY c_boolean, c_bigint, c_date; +---- +{f} {-32767} {-2147483647} {-9223372036854775807} +{} {} {} {} +NULL NULL NULL NULL +{NULL,t} {NULL,1} {NULL,123} {NULL,1234567890} +{f} {-32767} {-2147483647} {-9223372036854775807} +{} {} {} {} +NULL NULL NULL NULL +{NULL,t} {NULL,1} {NULL,123} {NULL,1234567890} + + +query TTTTTTT +SELECT c_decimal_array, c_real_array, c_double_precision_array +FROM postgres_all_types +ORDER BY c_boolean, c_bigint, c_date; +---- +{-10.0} {-10000} {-10000} +{} {} {} +NULL NULL NULL +{NULL,123.45} {NULL,123.45} {NULL,123.456} +{-10.0} {-10000} {-10000} +{} {} {} +NULL NULL NULL +{NULL,123.45} {NULL,123.45} {NULL,123.456} + +query TTTTTTT +SELECT c_varchar_array, c_bytea_array +FROM postgres_all_types +ORDER BY c_boolean, c_bigint, c_date; +---- +{""} {"\\x00"} +{} {} +NULL NULL +{NULL,example} {NULL,"\\xdeadbeef"} +{""} {"\\x00"} +{} {} +NULL NULL +{NULL,example} {NULL,"\\xdeadbeef"} + +query TTTTTTT +SELECT c_date_array, c_time_array, c_timestamp_array, c_timestamptz_array, c_interval_array +FROM postgres_all_types +ORDER BY c_boolean, c_bigint, c_date; +---- +{0001-01-01} {00:00:00} {"2001-01-01 00:00:00"} {"2001-01-01 08:00:00+00:00"} NULL +{} {} {} {} NULL +NULL NULL NULL NULL NULL +{NULL,2024-05-19} {NULL,12:34:56} {NULL,"2024-05-19 12:34:56"} {NULL,"2024-05-19 12:34:56+00:00"} NULL +{0001-01-01} {00:00:00} {"2001-01-01 00:00:00"} {"2001-01-01 08:00:00+00:00"} NULL +{} {} {} {} NULL +NULL NULL NULL NULL NULL +{NULL,2024-05-19} {NULL,12:34:56} {NULL,"2024-05-19 12:34:56"} {NULL,"2024-05-19 12:34:56+00:00"} NULL + +query TTTTTTT +SELECT c_jsonb_array, c_uuid_array, c_enum_array +FROM postgres_all_types +ORDER BY c_boolean, c_bigint, c_date; +---- +{"{}"} {bb488f9b-330d-4012-b849-12adeb49e57e} NULL +{} {} {} +NULL NULL NULL +{NULL,"{\"key\": \"value\"}"} {NULL,123e4567-e89b-12d3-a456-426614174000} NULL +{"{}"} {bb488f9b-330d-4012-b849-12adeb49e57e} NULL +{} {} {} +NULL NULL NULL +{NULL,"{\"key\": \"value\"}"} {NULL,123e4567-e89b-12d3-a456-426614174000} NULL diff --git a/e2e_test/source/cdc/cdc.share_stream.slt b/e2e_test/source/cdc/cdc.share_stream.slt index 3563ee6ed2a4b..e07a0c1d773ef 100644 --- a/e2e_test/source/cdc/cdc.share_stream.slt +++ b/e2e_test/source/cdc/cdc.share_stream.slt @@ -43,6 +43,9 @@ create table rw.products_test ( id INT, PRIMARY KEY (id) ) include timestamp as commit_ts from mysql_mytest table 'mytest.products'; +# sleep to ensure (default,'Milk','Milk is a white liquid food') is consumed from Debezium message instead of backfill. +sleep 10s + system ok mysql --protocol=tcp -u root mytest -e "INSERT INTO products VALUES (default,'Milk','Milk is a white liquid food'); INSERT INTO orders VALUES (default, '2023-11-28 15:08:22', 'Bob', 10.52, 100, false);" @@ -206,6 +209,8 @@ CREATE TABLE IF NOT EXISTS postgres_all_types( c_timestamptz timestamptz, c_interval interval, c_jsonb jsonb, + c_uuid varchar, + c_enum varchar, c_boolean_array boolean[], c_smallint_array smallint[], c_integer_array integer[], @@ -221,7 +226,8 @@ CREATE TABLE IF NOT EXISTS postgres_all_types( c_timestamptz_array timestamptz[], c_interval_array interval[], c_jsonb_array jsonb[], - c_uuid varchar, + c_uuid_array varchar[], + c_enum_array varchar[], PRIMARY KEY (c_boolean,c_bigint,c_date) ) from pg_source table 'public.postgres_all_types'; @@ -253,7 +259,7 @@ sleep 3s query TTTTTTT SELECT c_boolean,c_date,c_time,c_timestamp,c_jsonb,c_smallint_array,c_timestamp_array,c_uuid FROM postgres_all_types where c_bigint=-9223372036854775807 ---- -f 0001-01-01 00:00:00 0001-01-01 00:00:00 {} {-32767} {"0001-01-01 00:00:00"} bb488f9b-330d-4012-b849-12adeb49e57e +f 0001-01-01 00:00:00 2001-01-01 00:00:00 {} {-32767} {"2001-01-01 00:00:00"} bb488f9b-330d-4012-b849-12adeb49e57e # postgres streaming test diff --git a/e2e_test/source/cdc/postgres_cdc.sql b/e2e_test/source/cdc/postgres_cdc.sql index b936fb7876ade..6579bc2683037 100644 --- a/e2e_test/source/cdc/postgres_cdc.sql +++ b/e2e_test/source/cdc/postgres_cdc.sql @@ -35,6 +35,7 @@ create table abs.t1 ("V1" int primary key, v2 double precision, v3 varchar, v4 n create publication my_publicaton for table abs.t1 ("V1", v3); insert into abs.t1 values (1, 1.1, 'aaa', '5431.1234'); +CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy'); CREATE TABLE IF NOT EXISTS postgres_all_types( c_boolean boolean, @@ -52,6 +53,8 @@ CREATE TABLE IF NOT EXISTS postgres_all_types( c_timestamptz timestamptz, c_interval interval, c_jsonb jsonb, + c_uuid uuid, + c_enum mood, c_boolean_array boolean[], c_smallint_array smallint[], c_integer_array integer[], @@ -67,11 +70,14 @@ CREATE TABLE IF NOT EXISTS postgres_all_types( c_timestamptz_array timestamptz[], c_interval_array interval[], c_jsonb_array jsonb[], - c_uuid uuid, + c_uuid_array uuid[], + c_enum_array mood[], PRIMARY KEY (c_boolean,c_bigint,c_date) ); -INSERT INTO postgres_all_types VALUES ( False, 0, 0, 0, 0, 0, 0, '', '\x00', '0001-01-01', '00:00:00', '0001-01-01 00:00:00'::timestamp, '0001-01-01 00:00:00'::timestamptz, interval '0 second', '{}', array[]::boolean[], array[]::smallint[], array[]::integer[], array[]::bigint[], array[]::decimal[], array[]::real[], array[]::double precision[], array[]::varchar[], array[]::bytea[], array[]::date[], array[]::time[], array[]::timestamp[], array[]::timestamptz[], array[]::interval[], array[]::jsonb[], null); -INSERT INTO postgres_all_types VALUES ( False, -32767, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, '', '\x00', '0001-01-01', '00:00:00', '0001-01-01 00:00:00'::timestamp, '0001-01-01 00:00:00'::timestamptz, interval '0 second', '{}', array[False::boolean]::boolean[], array[-32767::smallint]::smallint[], array[-2147483647::integer]::integer[], array[-9223372036854775807::bigint]::bigint[], array[-10.0::decimal]::decimal[], array[-9999.999999::real]::real[], array[-10000.0::double precision]::double precision[], array[''::varchar]::varchar[], array['\x00'::bytea]::bytea[], array['0001-01-01'::date]::date[], array['00:00:00'::time]::time[], array['0001-01-01 00:00:00'::timestamp::timestamp]::timestamp[], array['0001-01-01 00:00:00'::timestamptz::timestamptz]::timestamptz[], array[interval '0 second'::interval]::interval[], array['{}'::jsonb]::jsonb[], 'bb488f9b-330d-4012-b849-12adeb49e57e'); +INSERT INTO postgres_all_types VALUES ( False, 0, 0, 0, 0, 0, 0, '', '\x00', '0001-01-01', '00:00:00', '2001-01-01 00:00:00'::timestamp, '2001-01-01 00:00:00-8'::timestamptz, interval '0 second', '{}', null, 'sad', array[]::boolean[], array[]::smallint[], array[]::integer[], array[]::bigint[], array[]::decimal[], array[]::real[], array[]::double precision[], array[]::varchar[], array[]::bytea[], array[]::date[], array[]::time[], array[]::timestamp[], array[]::timestamptz[], array[]::interval[], array[]::jsonb[], array[]::uuid[], array[]::mood[]); +INSERT INTO postgres_all_types VALUES ( False, -32767, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, 'd', '\x00', '0001-01-01', '00:00:00', '2001-01-01 00:00:00'::timestamp, '2001-01-01 00:00:00-8'::timestamptz, interval '0 second', '{}', 'bb488f9b-330d-4012-b849-12adeb49e57e', 'happy', array[False::boolean]::boolean[], array[-32767::smallint]::smallint[], array[-2147483647::integer]::integer[], array[-9223372036854775807::bigint]::bigint[], array[-10.0::decimal]::decimal[], array[-9999.999999::real]::real[], array[-10000.0::double precision]::double precision[], array[''::varchar]::varchar[], array['\x00'::bytea]::bytea[], array['0001-01-01'::date]::date[], array['00:00:00'::time]::time[], array['2001-01-01 00:00:00'::timestamp::timestamp]::timestamp[], array['2001-01-01 00:00:00-8'::timestamptz::timestamptz]::timestamptz[], array[interval '0 second'::interval]::interval[], array['{}'::jsonb]::jsonb[], '{bb488f9b-330d-4012-b849-12adeb49e57e}', '{happy,ok,NULL}'); +INSERT INTO postgres_all_types VALUES ( False, 1, 123, 1234567890, 123.45, 123.45, 123.456, 'example', '\xDEADBEEF', '0024-01-01', '12:34:56', '2024-05-19 12:34:56', '2024-05-19 12:34:56+00', INTERVAL '1 day', '{"key": "value"}', '123e4567-e89b-12d3-a456-426614174000', 'happy', ARRAY[NULL, TRUE]::boolean[], ARRAY[NULL, 1::smallint], ARRAY[NULL, 123], ARRAY[NULL, 1234567890], ARRAY[NULL, 123.45::numeric], ARRAY[NULL, 123.45::real], ARRAY[NULL, 123.456], ARRAY[NULL, 'example'], ARRAY[NULL, '\xDEADBEEF'::bytea], ARRAY[NULL, '2024-05-19'::date], ARRAY[NULL, '12:34:56'::time], ARRAY[NULL, '2024-05-19 12:34:56'::timestamp], ARRAY[NULL, '2024-05-19 12:34:56+00'::timestamptz], ARRAY[NULL, INTERVAL '1 day'], ARRAY[NULL, '{"key": "value"}'::jsonb], ARRAY[NULL, '123e4567-e89b-12d3-a456-426614174000'::uuid], ARRAY[NULL, 'happy'::mood]); +INSERT INTO postgres_all_types VALUES ( False, NULL, NULL, 1, NULL, NULL, NULL, NULL, NULL, '0024-05-19', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL); create table numeric_table(id int PRIMARY KEY, num numeric); insert into numeric_table values(1, 3.14); @@ -89,8 +95,6 @@ create table numeric_list(id int primary key, num numeric[]); insert into numeric_list values(1, '{3.14, 6, 57896044618658097711785492504343953926634992332820282019728792003956564819967, 57896044618658097711785492504343953926634992332820282019728792003956564819968, 115792089237316195423570985008687907853269984665640564039457584007913129639936.555555}'); insert into numeric_list values(2, '{nan, infinity, -infinity}'); ---- for https://github.com/risingwavelabs/risingwave/issues/16392 -CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy'); CREATE TABLE enum_table ( id int PRIMARY KEY, current_mood mood @@ -100,3 +104,4 @@ INSERT INTO enum_table VALUES (1, 'happy'); CREATE TABLE list_with_null(id int primary key, my_int int[], my_num numeric[], my_num_1 numeric[], my_num_2 numeric[], my_mood mood[], my_uuid uuid[], my_bytea bytea[]); INSERT INTO list_with_null VALUES (1, '{1,2,NULL}', '{1.1,inf,NULL}', '{1.1,inf,NULL}', '{1.1,inf,NULL}', '{happy,ok,NULL}', '{bb488f9b-330d-4012-b849-12adeb49e57e,bb488f9b-330d-4012-b849-12adeb49e57f, NULL}', '{\\x00,\\x01,NULL}'); INSERT INTO list_with_null VALUES (2, '{NULL,3,4}', '{2.2,0,NULL}' , '{2.2,0,NULL}', '{2.2,0,NULL}', '{happy,ok,sad}', '{2de296df-eda7-4202-a81f-1036100ef4f6,2977afbc-0b12-459c-a36f-f623fc9e9840}', '{\\x00,\\x01,\\x02}'); +INSERT INTO list_with_null VALUES (5, NULL, NULL, NULL, NULL, NULL, NULL, NULL); diff --git a/e2e_test/source/cdc/postgres_cdc_insert.sql b/e2e_test/source/cdc/postgres_cdc_insert.sql index a02a35a020965..4c0d0dee48b42 100644 --- a/e2e_test/source/cdc/postgres_cdc_insert.sql +++ b/e2e_test/source/cdc/postgres_cdc_insert.sql @@ -12,6 +12,11 @@ SELECT pg_current_wal_lsn(); select * from pg_publication_tables where pubname='rw_publication'; select * from public.person order by id; +INSERT INTO postgres_all_types VALUES ( True, 0, 0, 0, 0, 0, 0, '', '\x00', '0001-01-01', '00:00:00', '2001-01-01 00:00:00'::timestamp, '2001-01-01 00:00:00-8'::timestamptz, interval '0 second', '{}', null, 'sad', array[]::boolean[], array[]::smallint[], array[]::integer[], array[]::bigint[], array[]::decimal[], array[]::real[], array[]::double precision[], array[]::varchar[], array[]::bytea[], array[]::date[], array[]::time[], array[]::timestamp[], array[]::timestamptz[], array[]::interval[], array[]::jsonb[], array[]::uuid[], array[]::mood[]); +INSERT INTO postgres_all_types VALUES ( True, -32767, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, 'd', '\x00', '0001-01-01', '00:00:00', '2001-01-01 00:00:00'::timestamp, '2001-01-01 00:00:00-8'::timestamptz, interval '0 second', '{}', 'bb488f9b-330d-4012-b849-12adeb49e57e', 'happy', array[False::boolean]::boolean[], array[-32767::smallint]::smallint[], array[-2147483647::integer]::integer[], array[-9223372036854775807::bigint]::bigint[], array[-10.0::decimal]::decimal[], array[-9999.999999::real]::real[], array[-10000.0::double precision]::double precision[], array[''::varchar]::varchar[], array['\x00'::bytea]::bytea[], array['0001-01-01'::date]::date[], array['00:00:00'::time]::time[], array['2001-01-01 00:00:00'::timestamp::timestamp]::timestamp[], array['2001-01-01 00:00:00-8'::timestamptz::timestamptz]::timestamptz[], array[interval '0 second'::interval]::interval[], array['{}'::jsonb]::jsonb[], '{bb488f9b-330d-4012-b849-12adeb49e57e}', '{happy,ok,NULL}'); +INSERT INTO postgres_all_types VALUES ( True, 1, 123, 1234567890, 123.45, 123.45, 123.456, 'example', '\xDEADBEEF', '0024-01-01', '12:34:56', '2024-05-19 12:34:56', '2024-05-19 12:34:56+00', INTERVAL '1 day', '{"key": "value"}', '123e4567-e89b-12d3-a456-426614174000', 'happy', ARRAY[NULL, TRUE]::boolean[], ARRAY[NULL, 1::smallint], ARRAY[NULL, 123], ARRAY[NULL, 1234567890], ARRAY[NULL, 123.45::numeric], ARRAY[NULL, 123.45::real], ARRAY[NULL, 123.456], ARRAY[NULL, 'example'], ARRAY[NULL, '\xDEADBEEF'::bytea], ARRAY[NULL, '2024-05-19'::date], ARRAY[NULL, '12:34:56'::time], ARRAY[NULL, '2024-05-19 12:34:56'::timestamp], ARRAY[NULL, '2024-05-19 12:34:56+00'::timestamptz], ARRAY[NULL, INTERVAL '1 day'], ARRAY[NULL, '{"key": "value"}'::jsonb], ARRAY[NULL, '123e4567-e89b-12d3-a456-426614174000'::uuid], ARRAY[NULL, 'happy'::mood]); +INSERT INTO postgres_all_types VALUES ( True, NULL, NULL, 1, NULL, NULL, NULL, NULL, NULL, '0024-05-19', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL); + insert into numeric_table values(102, 57896044618658097711785492504343953926634992332820282019728792003956564819967); --- 2^255 insert into numeric_table values(103, 57896044618658097711785492504343953926634992332820282019728792003956564819968); @@ -25,3 +30,5 @@ INSERT INTO enum_table VALUES (3, 'sad'); --- to avoid escaping issues of psql -c "", we insert this row here and check the result in check_new_rows.slt INSERT INTO list_with_null VALUES (3, '{NULL,-3,-4}', '{NULL,nan,-inf}', '{NULL,nan,-inf}', '{NULL,nan,-inf}', '{NULL,sad,ok}', '{NULL,471acecf-a4b4-4ed3-a211-7fb2291f159f,9bc35adf-fb11-4130-944c-e7eadb96b829}', '{NULL,\\x99,\\xAA}'); INSERT INTO list_with_null VALUES (4, '{-4,-5,-6}', '{NULL,-99999999999999999.9999}', '{NULL,-99999999999999999.9999}', '{NULL,-99999999999999999.9999}', '{NULL,sad,ok}', '{b2e4636d-fa03-4ad4-bf16-029a79dca3e2}', '{\\x88,\\x99,\\xAA}'); +INSERT INTO list_with_null VALUES (6, NULL, NULL, NULL, NULL, NULL, NULL, NULL); + diff --git a/src/common/src/types/datetime.rs b/src/common/src/types/datetime.rs index 9199dd41b6e5b..7058d36ec6fd5 100644 --- a/src/common/src/types/datetime.rs +++ b/src/common/src/types/datetime.rs @@ -22,7 +22,7 @@ use std::str::FromStr; use bytes::{Bytes, BytesMut}; use chrono::{Datelike, Days, Duration, NaiveDate, NaiveDateTime, NaiveTime, Timelike, Weekday}; -use postgres_types::{accepts, to_sql_checked, IsNull, ToSql, Type}; +use postgres_types::{accepts, to_sql_checked, FromSql, IsNull, ToSql, Type}; use risingwave_common_estimate_size::ZeroHeapSize; use thiserror::Error; @@ -88,6 +88,20 @@ impl ToSql for Date { } } +impl<'a> FromSql<'a> for Date { + fn from_sql( + ty: &Type, + raw: &'a [u8], + ) -> std::result::Result> { + let instant = NaiveDate::from_sql(ty, raw)?; + Ok(Self::from(instant)) + } + + fn accepts(ty: &Type) -> bool { + matches!(*ty, Type::DATE) + } +} + impl ToSql for Time { accepts!(TIME); @@ -105,6 +119,20 @@ impl ToSql for Time { } } +impl<'a> FromSql<'a> for Time { + fn from_sql( + ty: &Type, + raw: &'a [u8], + ) -> std::result::Result> { + let instant = NaiveTime::from_sql(ty, raw)?; + Ok(Self::from(instant)) + } + + fn accepts(ty: &Type) -> bool { + matches!(*ty, Type::TIME) + } +} + impl ToSql for Timestamp { accepts!(TIMESTAMP); @@ -122,6 +150,20 @@ impl ToSql for Timestamp { } } +impl<'a> FromSql<'a> for Timestamp { + fn from_sql( + ty: &Type, + raw: &'a [u8], + ) -> std::result::Result> { + let instant = NaiveDateTime::from_sql(ty, raw)?; + Ok(Self::from(instant)) + } + + fn accepts(ty: &Type) -> bool { + matches!(*ty, Type::TIMESTAMP) + } +} + /// Parse a date from varchar. /// /// # Example diff --git a/src/common/src/types/from_sql.rs b/src/common/src/types/from_sql.rs new file mode 100644 index 0000000000000..ba1d49892c602 --- /dev/null +++ b/src/common/src/types/from_sql.rs @@ -0,0 +1,66 @@ +// Copyright 2024 RisingWave Labs +// +// 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 postgres_types::{FromSql, Type}; +use risingwave_common::types::{ + Date, Interval, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, +}; + +impl<'a> FromSql<'a> for ScalarImpl { + fn from_sql( + ty: &Type, + raw: &'a [u8], + ) -> Result> { + Ok(match *ty { + Type::BOOL => ScalarImpl::from(bool::from_sql(ty, raw)?), + Type::INT2 => ScalarImpl::from(i16::from_sql(ty, raw)?), + Type::INT4 => ScalarImpl::from(i32::from_sql(ty, raw)?), + Type::INT8 => ScalarImpl::from(i64::from_sql(ty, raw)?), + Type::FLOAT4 => ScalarImpl::from(f32::from_sql(ty, raw)?), + Type::FLOAT8 => ScalarImpl::from(f64::from_sql(ty, raw)?), + Type::DATE => ScalarImpl::from(Date::from_sql(ty, raw)?), + Type::TIME => ScalarImpl::from(Time::from_sql(ty, raw)?), + Type::TIMESTAMP => ScalarImpl::from(Timestamp::from_sql(ty, raw)?), + Type::TIMESTAMPTZ => ScalarImpl::from(Timestamptz::from_sql(ty, raw)?), + Type::JSONB => ScalarImpl::from(JsonbVal::from_sql(ty, raw)?), + Type::INTERVAL => ScalarImpl::from(Interval::from_sql(ty, raw)?), + Type::BYTEA => ScalarImpl::from(Vec::::from_sql(ty, raw)?.into_boxed_slice()), + Type::VARCHAR | Type::TEXT => ScalarImpl::from(String::from_sql(ty, raw)?), + // Serial, Int256, Struct, List and Decimal are not supported here + // Note: The Decimal type is specially handled in the `ScalarAdapter`. + _ => bail_not_implemented!("the postgres decoding for {ty} is unsupported"), + }) + } + + fn accepts(ty: &Type) -> bool { + matches!( + *ty, + Type::BOOL + | Type::INT2 + | Type::INT4 + | Type::INT8 + | Type::FLOAT4 + | Type::FLOAT8 + | Type::DATE + | Type::TIME + | Type::TIMESTAMP + | Type::TIMESTAMPTZ + | Type::JSONB + | Type::INTERVAL + | Type::BYTEA + | Type::VARCHAR + | Type::TEXT + ) + } +} diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 522ec788d8646..824020fac3123 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt; +use std::fmt::{self, Write}; use std::hash::Hash; -use bytes::Buf; +use bytes::{Buf, BufMut, BytesMut}; use jsonbb::{Value, ValueRef}; +use postgres_types::{accepts, to_sql_checked, FromSql, IsNull, ToSql, Type}; use risingwave_common_estimate_size::EstimateSize; use super::{Datum, IntoOrdered, ListValue, ScalarImpl, StructRef, ToOwnedDatum, F64}; @@ -539,3 +540,38 @@ impl std::io::Write for FmtToIoUnchecked { Ok(()) } } + +impl ToSql for JsonbVal { + accepts!(JSONB); + + to_sql_checked!(); + + fn to_sql( + &self, + _ty: &Type, + out: &mut BytesMut, + ) -> Result> + where + Self: Sized, + { + out.put_u8(1); + write!(out, "{}", self.0).unwrap(); + Ok(IsNull::No) + } +} + +impl<'a> FromSql<'a> for JsonbVal { + fn from_sql( + _ty: &Type, + mut raw: &'a [u8], + ) -> Result> { + if raw.is_empty() || raw.get_u8() != 1 { + return Err("invalid jsonb encoding".into()); + } + Ok(JsonbVal::from(Value::from_text(raw)?)) + } + + fn accepts(ty: &Type) -> bool { + matches!(*ty, Type::JSONB) + } +} diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index bb41e75c8a4f8..01364fb15da63 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -48,6 +48,7 @@ use crate::{ mod datetime; mod decimal; mod fields; +mod from_sql; mod interval; mod jsonb; mod macros; diff --git a/src/common/src/types/timestamptz.rs b/src/common/src/types/timestamptz.rs index 1da5f0abd718f..feafee6e212bf 100644 --- a/src/common/src/types/timestamptz.rs +++ b/src/common/src/types/timestamptz.rs @@ -19,7 +19,7 @@ use std::str::FromStr; use bytes::{Bytes, BytesMut}; use chrono::{DateTime, Datelike, TimeZone, Utc}; use chrono_tz::Tz; -use postgres_types::{accepts, to_sql_checked, IsNull, ToSql, Type}; +use postgres_types::{accepts, to_sql_checked, FromSql, IsNull, ToSql, Type}; use risingwave_common_estimate_size::ZeroHeapSize; use serde::{Deserialize, Serialize}; @@ -51,6 +51,20 @@ impl ToSql for Timestamptz { } } +impl<'a> FromSql<'a> for Timestamptz { + fn from_sql( + ty: &Type, + raw: &'a [u8], + ) -> Result> { + let instant = DateTime::::from_sql(ty, raw)?; + Ok(Self::from(instant)) + } + + fn accepts(ty: &Type) -> bool { + matches!(*ty, Type::TIMESTAMPTZ) + } +} + impl ToBinary for Timestamptz { fn to_binary_with_type(&self, _ty: &DataType) -> super::to_binary::Result> { let instant = self.to_datetime_utc(); diff --git a/src/common/src/types/to_sql.rs b/src/common/src/types/to_sql.rs index 71957b3bf35c8..3ece8a574c450 100644 --- a/src/common/src/types/to_sql.rs +++ b/src/common/src/types/to_sql.rs @@ -15,11 +15,11 @@ use std::error::Error; use bytes::BytesMut; -use postgres_types::{accepts, to_sql_checked, IsNull, ToSql, Type}; +use postgres_types::{to_sql_checked, IsNull, ToSql, Type}; -use crate::types::{JsonbRef, ScalarRefImpl}; +use crate::types::ScalarImpl; -impl ToSql for ScalarRefImpl<'_> { +impl ToSql for ScalarImpl { to_sql_checked!(); fn to_sql(&self, ty: &Type, out: &mut BytesMut) -> Result> @@ -27,25 +27,23 @@ impl ToSql for ScalarRefImpl<'_> { Self: Sized, { match self { - ScalarRefImpl::Int16(v) => v.to_sql(ty, out), - ScalarRefImpl::Int32(v) => v.to_sql(ty, out), - ScalarRefImpl::Int64(v) => v.to_sql(ty, out), - ScalarRefImpl::Serial(v) => v.to_sql(ty, out), - ScalarRefImpl::Float32(v) => v.to_sql(ty, out), - ScalarRefImpl::Float64(v) => v.to_sql(ty, out), - ScalarRefImpl::Utf8(v) => v.to_sql(ty, out), - ScalarRefImpl::Bool(v) => v.to_sql(ty, out), - ScalarRefImpl::Decimal(v) => v.to_sql(ty, out), - ScalarRefImpl::Interval(v) => v.to_sql(ty, out), - ScalarRefImpl::Date(v) => v.to_sql(ty, out), - ScalarRefImpl::Timestamp(v) => v.to_sql(ty, out), - ScalarRefImpl::Timestamptz(v) => v.to_sql(ty, out), - ScalarRefImpl::Time(v) => v.to_sql(ty, out), - ScalarRefImpl::Bytea(v) => v.to_sql(ty, out), - ScalarRefImpl::Jsonb(_) // jsonbb::Value doesn't implement ToSql yet - | ScalarRefImpl::Int256(_) - | ScalarRefImpl::Struct(_) - | ScalarRefImpl::List(_) => { + ScalarImpl::Int16(v) => v.to_sql(ty, out), + ScalarImpl::Int32(v) => v.to_sql(ty, out), + ScalarImpl::Int64(v) => v.to_sql(ty, out), + ScalarImpl::Serial(v) => v.to_sql(ty, out), + ScalarImpl::Float32(v) => v.to_sql(ty, out), + ScalarImpl::Float64(v) => v.to_sql(ty, out), + ScalarImpl::Utf8(v) => v.to_sql(ty, out), + ScalarImpl::Bool(v) => v.to_sql(ty, out), + ScalarImpl::Decimal(v) => v.to_sql(ty, out), + ScalarImpl::Interval(v) => v.to_sql(ty, out), + ScalarImpl::Date(v) => v.to_sql(ty, out), + ScalarImpl::Timestamp(v) => v.to_sql(ty, out), + ScalarImpl::Timestamptz(v) => v.to_sql(ty, out), + ScalarImpl::Time(v) => v.to_sql(ty, out), + ScalarImpl::Bytea(v) => (&**v).to_sql(ty, out), + ScalarImpl::Jsonb(v) => v.to_sql(ty, out), + ScalarImpl::Int256(_) | ScalarImpl::Struct(_) | ScalarImpl::List(_) => { bail_not_implemented!("the postgres encoding for {ty} is unsupported") } } @@ -59,18 +57,3 @@ impl ToSql for ScalarRefImpl<'_> { true } } - -impl ToSql for JsonbRef<'_> { - accepts!(JSONB); - - to_sql_checked!(); - - fn to_sql(&self, _: &Type, out: &mut BytesMut) -> Result> - where - Self: Sized, - { - let buf = self.value_serialize(); - out.extend(buf); - Ok(IsNull::No) - } -} diff --git a/src/connector/src/parser/postgres.rs b/src/connector/src/parser/postgres.rs index f01484ad18fb2..da17ea256ba3c 100644 --- a/src/connector/src/parser/postgres.rs +++ b/src/connector/src/parser/postgres.rs @@ -14,54 +14,17 @@ use std::sync::LazyLock; -use chrono::{NaiveDate, Utc}; use risingwave_common::catalog::Schema; use risingwave_common::log::LogSuppresser; use risingwave_common::row::OwnedRow; -use risingwave_common::types::{ - DataType, Date, Decimal, Interval, JsonbVal, ListValue, ScalarImpl, Time, Timestamp, - Timestamptz, -}; +use risingwave_common::types::{DataType, Decimal, ScalarImpl}; use thiserror_ext::AsReport; -use tokio_postgres::types::{Kind, Type}; use crate::parser::scalar_adapter::ScalarAdapter; use crate::parser::util::log_error; static LOG_SUPPERSSER: LazyLock = LazyLock::new(LogSuppresser::default); -macro_rules! handle_list_data_type { - ($row:expr, $i:expr, $name:expr, $type:ty, $builder:expr) => { - let res = $row.try_get::<_, Option>>>($i); - match res { - Ok(val) => { - if let Some(v) = val { - v.into_iter() - .for_each(|val| $builder.append(val.map(ScalarImpl::from))) - } - } - Err(err) => { - log_error!($name, err, "parse column failed"); - } - } - }; - ($row:expr, $i:expr, $name:expr, $type:ty, $builder:expr, $rw_type:ty) => { - let res = $row.try_get::<_, Option>>>($i); - match res { - Ok(val) => { - if let Some(v) = val { - v.into_iter().for_each(|val| { - $builder.append(val.map(|v| ScalarImpl::from(<$rw_type>::from(v)))) - }) - } - } - Err(err) => { - log_error!($name, err, "parse column failed"); - } - } - }; -} - macro_rules! handle_data_type { ($row:expr, $i:expr, $name:expr, $type:ty) => {{ let res = $row.try_get::<_, Option<$type>>($i); @@ -73,16 +36,6 @@ macro_rules! handle_data_type { } } }}; - ($row:expr, $i:expr, $name:expr, $type:ty, $rw_type:ty) => {{ - let res = $row.try_get::<_, Option<$type>>($i); - match res { - Ok(val) => val.map(|v| ScalarImpl::from(<$rw_type>::from(v))), - Err(err) => { - log_error!($name, err, "parse column failed"); - None - } - } - }}; } pub fn postgres_row_to_owned_row(row: tokio_postgres::Row, schema: &Schema) -> OwnedRow { @@ -106,304 +59,65 @@ fn postgres_cell_to_scalar_impl( // Issue #1. The null of enum list is not supported in Debezium. An enum list contains `NULL` will fallback to `NULL`. // Issue #2. In our parser, when there's inf, -inf, nan or invalid item in a list, the whole list will fallback null. match data_type { - DataType::Boolean => { - handle_data_type!(row, i, name, bool) - } - DataType::Int16 => { - handle_data_type!(row, i, name, i16) - } - DataType::Int32 => { - handle_data_type!(row, i, name, i32) - } - DataType::Int64 => { - handle_data_type!(row, i, name, i64) - } - DataType::Float32 => { - handle_data_type!(row, i, name, f32) - } - DataType::Float64 => { - handle_data_type!(row, i, name, f64) - } - DataType::Decimal => { - handle_data_type!(row, i, name, Decimal) - } - DataType::Int256 => { - // Currently in order to handle the decimal beyond RustDecimal, - // we use the PgNumeric type to convert the decimal to a string. - // Then we convert the string to Int256. - // Note: It's only used to map the numeric type in upstream Postgres to RisingWave's rw_int256. - let res = row.try_get::<_, Option>>(i); + DataType::Boolean + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Date + | DataType::Time + | DataType::Timestamp + | DataType::Timestamptz + | DataType::Jsonb + | DataType::Interval + | DataType::Bytea => { + // ScalarAdapter is also fine. But ScalarImpl is more efficient + let res = row.try_get::<_, Option>(i); match res { - Ok(val) => val.and_then(|v| v.into_scalar(DataType::Int256)), + Ok(val) => val, Err(err) => { - log_error!(name, err, "parse numeric column as pg_numeric failed"); + log_error!(name, err, "parse column failed"); None } } } - DataType::Varchar => { - if let Kind::Enum(_) = row.columns()[i].type_().kind() { - // enum type needs to be handled separately - let res = row.try_get::<_, Option>>(i); - match res { - Ok(val) => val.and_then(|v| v.into_scalar(DataType::Varchar)), - Err(err) => { - log_error!(name, err, "parse enum column failed"); - None - } - } - } else { - match *row.columns()[i].type_() { - // Since we don't support UUID natively, adapt it to a VARCHAR column - Type::UUID => { - let res = row.try_get::<_, Option>>(i); - match res { - Ok(val) => val.and_then(|v| v.into_scalar(DataType::Varchar)), - Err(err) => { - log_error!(name, err, "parse uuid column failed"); - None - } - } - } - // we support converting NUMERIC to VARCHAR implicitly - Type::NUMERIC => { - // Currently in order to handle the decimal beyond RustDecimal, - // we use the PgNumeric type to convert the decimal to a string. - // Note: It's only used to map the numeric type in upstream Postgres to RisingWave's varchar. - let res = row.try_get::<_, Option>>(i); - match res { - Ok(val) => val.and_then(|v| v.into_scalar(DataType::Varchar)), - Err(err) => { - log_error!(name, err, "parse numeric column as pg_numeric failed"); - None - } - } - } - _ => { - handle_data_type!(row, i, name, String) - } - } - } - } - DataType::Date => { - handle_data_type!(row, i, name, NaiveDate, Date) - } - DataType::Time => { - handle_data_type!(row, i, name, chrono::NaiveTime, Time) - } - DataType::Timestamp => { - handle_data_type!(row, i, name, chrono::NaiveDateTime, Timestamp) - } - DataType::Timestamptz => { - handle_data_type!(row, i, name, chrono::DateTime, Timestamptz) + DataType::Decimal => { + // Decimal is more efficient than PgNumeric in ScalarAdapter + handle_data_type!(row, i, name, Decimal) } - DataType::Bytea => { - let res = row.try_get::<_, Option>>(i); + DataType::Varchar | DataType::Int256 => { + let res = row.try_get::<_, Option>(i); match res { - Ok(val) => val.map(|v| ScalarImpl::from(v.into_boxed_slice())), + Ok(val) => val.and_then(|v| v.into_scalar(data_type)), Err(err) => { log_error!(name, err, "parse column failed"); None } } } - DataType::Jsonb => { - handle_data_type!(row, i, name, serde_json::Value, JsonbVal) - } - DataType::Interval => { - handle_data_type!(row, i, name, Interval) - } - DataType::List(dtype) => { - let mut builder = dtype.create_array_builder(0); - // enum list needs to be handled separately - if let Kind::Array(item_type) = row.columns()[i].type_().kind() - && let Kind::Enum(_) = item_type.kind() - { - // Issue #1, we use ScalarAdapter instead of Option - let res = row.try_get::<_, Option>>>(i); + DataType::List(dtype) => match **dtype { + // TODO(Kexiang): allow DataType::List(_) + DataType::Struct(_) | DataType::List(_) | DataType::Serial => { + tracing::warn!( + "unsupported List data type {:?}, set the List to empty", + **dtype + ); + None + } + _ => { + let res = row.try_get::<_, Option>(i); match res { - Ok(val) => { - if let Some(vec) = val { - for val in vec { - builder.append(val.into_scalar(DataType::Varchar)) - } - } - Some(ScalarImpl::from(ListValue::new(builder.finish()))) - } + Ok(val) => val.and_then(|v| v.into_scalar(data_type)), Err(err) => { - log_error!(name, err, "parse enum column failed"); + log_error!(name, err, "parse list column failed"); None } } - } else { - match **dtype { - DataType::Boolean => { - handle_list_data_type!(row, i, name, bool, builder); - } - DataType::Int16 => { - handle_list_data_type!(row, i, name, i16, builder); - } - DataType::Int32 => { - handle_list_data_type!(row, i, name, i32, builder); - } - DataType::Int64 => { - handle_list_data_type!(row, i, name, i64, builder); - } - DataType::Float32 => { - handle_list_data_type!(row, i, name, f32, builder); - } - DataType::Float64 => { - handle_list_data_type!(row, i, name, f64, builder); - } - DataType::Decimal => { - let res = row.try_get::<_, Option>>>>(i); - match res { - Ok(val) => { - if let Some(vec) = val { - builder = ScalarAdapter::build_scalar_in_list( - vec, - DataType::Decimal, - builder, - )?; - } - } - Err(err) => { - log_error!(name, err, "parse uuid column failed"); - } - }; - } - DataType::Date => { - handle_list_data_type!(row, i, name, NaiveDate, builder, Date); - } - DataType::Varchar => { - match *row.columns()[i].type_() { - // Since we don't support UUID natively, adapt it to a VARCHAR column - Type::UUID_ARRAY => { - let res = - row.try_get::<_, Option>>>>(i); - match res { - Ok(val) => { - if let Some(vec) = val { - for val in vec { - builder.append( - val.and_then(|v| { - v.into_scalar(DataType::Varchar) - }), - ) - } - } - } - Err(err) => { - log_error!(name, err, "parse uuid column failed"); - } - }; - } - Type::NUMERIC_ARRAY => { - let res = - row.try_get::<_, Option>>>>(i); - match res { - Ok(val) => { - if let Some(vec) = val { - builder = ScalarAdapter::build_scalar_in_list( - vec, - DataType::Varchar, - builder, - )?; - } - } - Err(err) => { - log_error!( - name, - err, - "parse numeric list column as pg_numeric list failed" - ); - } - }; - } - _ => { - handle_list_data_type!(row, i, name, String, builder); - } - } - } - DataType::Time => { - handle_list_data_type!(row, i, name, chrono::NaiveTime, builder, Time); - } - DataType::Timestamp => { - handle_list_data_type!( - row, - i, - name, - chrono::NaiveDateTime, - builder, - Timestamp - ); - } - DataType::Timestamptz => { - handle_list_data_type!( - row, - i, - name, - chrono::DateTime, - builder, - Timestamptz - ); - } - DataType::Interval => { - handle_list_data_type!(row, i, name, Interval, builder); - } - DataType::Jsonb => { - handle_list_data_type!(row, i, name, serde_json::Value, builder, JsonbVal); - } - DataType::Bytea => { - let res = row.try_get::<_, Option>>>>(i); - match res { - Ok(val) => { - if let Some(v) = val { - v.into_iter().for_each(|val| { - builder.append( - val.map(|v| ScalarImpl::from(v.into_boxed_slice())), - ) - }) - } - } - Err(err) => { - log_error!(name, err, "parse column failed"); - } - } - } - DataType::Int256 => { - let res = row.try_get::<_, Option>>>>(i); - match res { - Ok(val) => { - if let Some(vec) = val { - builder = ScalarAdapter::build_scalar_in_list( - vec, - DataType::Int256, - builder, - )?; - } - } - Err(err) => { - log_error!( - name, - err, - "parse numeric list column as pg_numeric list failed" - ); - } - }; - } - DataType::Struct(_) | DataType::List(_) | DataType::Serial => { - tracing::warn!( - "unsupported List data type {:?}, set the List to empty", - **dtype - ); - } - }; - Some(ScalarImpl::from(ListValue::new(builder.finish()))) } - } + }, DataType::Struct(_) | DataType::Serial => { - // Interval and Struct are not supported + // Struct and Serial are not supported tracing::warn!(name, ?data_type, "unsupported data type, set to null"); None } diff --git a/src/connector/src/parser/scalar_adapter.rs b/src/connector/src/parser/scalar_adapter.rs index 6a6546a8d7600..0f5d2d6d6d935 100644 --- a/src/connector/src/parser/scalar_adapter.rs +++ b/src/connector/src/parser/scalar_adapter.rs @@ -17,7 +17,7 @@ use std::str::FromStr; use anyhow::anyhow; use bytes::BytesMut; use pg_bigdecimal::PgNumeric; -use risingwave_common::types::{DataType, Decimal, Int256, ScalarImpl, ScalarRefImpl}; +use risingwave_common::types::{DataType, Decimal, Int256, ListValue, ScalarImpl, ScalarRefImpl}; use thiserror_ext::AsReport; use tokio_postgres::types::{to_sql_checked, FromSql, IsNull, Kind, ToSql, Type}; @@ -75,14 +75,21 @@ impl ToSql for EnumString { /// Adapter for `ScalarImpl` to Postgres data type, /// which can be used to encode/decode to/from Postgres value. #[derive(Debug)] -pub(crate) enum ScalarAdapter<'a> { - Builtin(ScalarRefImpl<'a>), +pub(crate) enum ScalarAdapter { + Builtin(ScalarImpl), Uuid(uuid::Uuid), + // Currently in order to handle the decimal beyond RustDecimal, + // we use the PgNumeric type to convert the decimal to a string/decimal/rw_int256. Numeric(PgNumeric), Enum(EnumString), + NumericList(Vec>), + EnumList(Vec>), + // UuidList is covered by List, while NumericList and EnumList are special cases. + // Note: The IntervalList is not supported. + List(Vec>), } -impl ToSql for ScalarAdapter<'_> { +impl ToSql for ScalarAdapter { to_sql_checked!(); fn to_sql( @@ -95,6 +102,9 @@ impl ToSql for ScalarAdapter<'_> { ScalarAdapter::Uuid(v) => v.to_sql(ty, out), ScalarAdapter::Numeric(v) => v.to_sql(ty, out), ScalarAdapter::Enum(v) => v.to_sql(ty, out), + ScalarAdapter::NumericList(v) => v.to_sql(ty, out), + ScalarAdapter::EnumList(v) => v.to_sql(ty, out), + ScalarAdapter::List(v) => v.to_sql(ty, out), } } @@ -104,133 +114,204 @@ impl ToSql for ScalarAdapter<'_> { } /// convert from Postgres uuid, numeric and enum to `ScalarAdapter` -impl<'a> FromSql<'a> for ScalarAdapter<'_> { +impl<'a> FromSql<'a> for ScalarAdapter { fn from_sql( ty: &Type, raw: &'a [u8], ) -> Result> { match ty.kind() { Kind::Simple => match *ty { - Type::UUID => { - let uuid = uuid::Uuid::from_sql(ty, raw)?; - Ok(ScalarAdapter::Uuid(uuid)) - } - Type::NUMERIC => { - let numeric = PgNumeric::from_sql(ty, raw)?; - Ok(ScalarAdapter::Numeric(numeric)) - } - _ => Err(anyhow!("failed to convert type {:?} to ScalarAdapter", ty).into()), + Type::UUID => Ok(ScalarAdapter::Uuid(uuid::Uuid::from_sql(ty, raw)?)), + // In order to cover the decimal beyond RustDecimal(only 28 digits are supported), + // we use the PgNumeric to handle decimal from postgres. + Type::NUMERIC => Ok(ScalarAdapter::Numeric(PgNumeric::from_sql(ty, raw)?)), + _ => Ok(ScalarAdapter::Builtin(ScalarImpl::from_sql(ty, raw)?)), }, Kind::Enum(_) => Ok(ScalarAdapter::Enum(EnumString::from_sql(ty, raw)?)), + Kind::Array(Type::NUMERIC) => { + Ok(ScalarAdapter::NumericList(FromSql::from_sql(ty, raw)?)) + } + Kind::Array(inner_type) if let Kind::Enum(_) = inner_type.kind() => { + Ok(ScalarAdapter::EnumList(FromSql::from_sql(ty, raw)?)) + } + Kind::Array(_) => Ok(ScalarAdapter::List(FromSql::from_sql(ty, raw)?)), _ => Err(anyhow!("failed to convert type {:?} to ScalarAdapter", ty).into()), } } fn accepts(ty: &Type) -> bool { - matches!(ty, &Type::UUID | &Type::NUMERIC) || ::accepts(ty) + match ty.kind() { + Kind::Simple => { + matches!(ty, &Type::UUID | &Type::NUMERIC) || ::accepts(ty) + } + Kind::Enum(_) => true, + Kind::Array(inner_type) => ::accepts(inner_type), + _ => false, + } } } -impl ScalarAdapter<'_> { +impl ScalarAdapter { pub fn name(&self) -> &'static str { match self { ScalarAdapter::Builtin(_) => "Builtin", ScalarAdapter::Uuid(_) => "Uuid", ScalarAdapter::Numeric(_) => "Numeric", ScalarAdapter::Enum(_) => "Enum", + ScalarAdapter::EnumList(_) => "EnumList", + ScalarAdapter::NumericList(_) => "NumericList", + ScalarAdapter::List(_) => "List", } } /// convert `ScalarRefImpl` to `ScalarAdapter` so that we can correctly encode to postgres value - pub(crate) fn from_scalar<'a>( - scalar: ScalarRefImpl<'a>, + pub(crate) fn from_scalar( + scalar: ScalarRefImpl<'_>, ty: &Type, - ) -> ConnectorResult> { + ) -> ConnectorResult { Ok(match (scalar, ty, ty.kind()) { (ScalarRefImpl::Utf8(s), &Type::UUID, _) => ScalarAdapter::Uuid(s.parse()?), (ScalarRefImpl::Utf8(s), &Type::NUMERIC, _) => { - ScalarAdapter::Numeric(string_to_pg_numeric(s)?) + ScalarAdapter::Numeric(string_to_pg_numeric(s)) } (ScalarRefImpl::Int256(s), &Type::NUMERIC, _) => { - ScalarAdapter::Numeric(string_to_pg_numeric(&s.to_string())?) + ScalarAdapter::Numeric(string_to_pg_numeric(&s.to_string())) } (ScalarRefImpl::Utf8(s), _, Kind::Enum(_)) => { ScalarAdapter::Enum(EnumString(s.to_owned())) } - _ => ScalarAdapter::Builtin(scalar), + (ScalarRefImpl::List(list), &Type::NUMERIC_ARRAY, _) => { + let mut vec = vec![]; + for datum in list.iter() { + vec.push(match datum { + Some(ScalarRefImpl::Int256(s)) => Some(string_to_pg_numeric(&s.to_string())), + Some(ScalarRefImpl::Decimal(s)) => Some(rw_numeric_to_pg_numeric(s)), + Some(ScalarRefImpl::Utf8(s)) => Some(string_to_pg_numeric(s)), + None => None, + _ => { + unreachable!("Only rw-numeric[], rw_int256[] and varchar[] are supported to convert to pg-numeric[]"); + } + }) + } + ScalarAdapter::NumericList(vec) + } + (ScalarRefImpl::List(list), _, Kind::Array(inner_type)) => match inner_type.kind() { + Kind::Enum(_) => { + let mut vec = vec![]; + for datum in list.iter() { + vec.push(match datum { + Some(ScalarRefImpl::Utf8(s)) => Some(EnumString(s.to_owned())), + _ => unreachable!( + "Only non-null varchar[] is supported to convert to enum[]" + ), + }) + } + ScalarAdapter::EnumList(vec) + } + _ => { + let mut vec = vec![]; + for datum in list.iter() { + vec.push( + datum + .map(|s| ScalarAdapter::from_scalar(s, inner_type)) + .transpose()?, + ); + } + ScalarAdapter::List(vec) + } + }, + _ => ScalarAdapter::Builtin(scalar.into_scalar_impl()), }) } - pub fn into_scalar(self, ty: DataType) -> Option { - match (&self, &ty) { - (ScalarAdapter::Builtin(scalar), _) => Some(scalar.into_scalar_impl()), + pub fn into_scalar(self, ty: &DataType) -> Option { + match (self, &ty) { + (ScalarAdapter::Builtin(scalar), _) => Some(scalar), (ScalarAdapter::Uuid(uuid), &DataType::Varchar) => { Some(ScalarImpl::from(uuid.to_string())) } (ScalarAdapter::Numeric(numeric), &DataType::Varchar) => { - Some(ScalarImpl::from(pg_numeric_to_string(numeric))) + Some(ScalarImpl::from(pg_numeric_to_string(&numeric))) } (ScalarAdapter::Numeric(numeric), &DataType::Int256) => { - pg_numeric_to_rw_int256(numeric) + pg_numeric_to_rw_int256(&numeric) } - (ScalarAdapter::Enum(EnumString(s)), &DataType::Varchar) => Some(ScalarImpl::from(s)), - _ => { - tracing::error!( - adapter = self.name(), - rw_type = ty.pg_name(), - "failed to convert from ScalarAdapter: invalid conversion" - ); - None + (ScalarAdapter::Numeric(numeric), &DataType::Decimal) => { + pg_numeric_to_rw_numeric(&numeric) } - } - } - - pub fn build_scalar_in_list( - vec: Vec>>, - ty: DataType, - mut builder: risingwave_common::array::ArrayBuilderImpl, - ) -> Option { - for val in vec { - let scalar = match (val, &ty) { - (Some(ScalarAdapter::Numeric(numeric)), &DataType::Varchar) => { - if pg_numeric_is_special(&numeric) { - return None; - } else { - Some(ScalarImpl::from(pg_numeric_to_string(&numeric))) - } - } - (Some(ScalarAdapter::Numeric(numeric)), &DataType::Int256) => match numeric { - PgNumeric::Normalized(big_decimal) => { - match Int256::from_str(big_decimal.to_string().as_str()) { - Ok(num) => Some(ScalarImpl::from(num)), - Err(err) => { - tracing::error!(error = %err.as_report(), "parse pg-numeric as rw_int256 failed"); + (ScalarAdapter::Enum(EnumString(s)), &DataType::Varchar) => Some(ScalarImpl::from(s)), + (ScalarAdapter::NumericList(vec), &DataType::List(dtype)) => { + let mut builder = dtype.create_array_builder(0); + for val in vec { + let scalar = match (val, &dtype) { + // A numeric array contains special values like NaN, Inf, -Inf, which are not supported in Debezium, + // when we encounter these special values, we fallback the array to NULL, returning None directly. + (Some(numeric), box DataType::Varchar) => { + if pg_numeric_is_special(&numeric) { return None; + } else { + ScalarAdapter::Numeric(numeric).into_scalar(dtype) } } - } - _ => return None, - }, - (Some(ScalarAdapter::Numeric(numeric)), &DataType::Decimal) => match numeric { - PgNumeric::Normalized(big_decimal) => { - match Decimal::from_str(big_decimal.to_string().as_str()) { - Ok(num) => Some(ScalarImpl::from(num)), - Err(err) => { - tracing::error!(error = %err.as_report(), "parse pg-numeric as rw-numeric failed (likely out-of-range"); + (Some(numeric), box DataType::Int256 | box DataType::Decimal) => { + if pg_numeric_is_special(&numeric) { return None; + } else { + // A PgNumeric can sometimes exceeds the range of Int256 and RwNumeric. + // In our json parsing, we fallback the array to NULL in this case. + // Here we keep the behavior consistent and return None directly. + match ScalarAdapter::Numeric(numeric).into_scalar(dtype) { + Some(scalar) => Some(scalar), + None => { + return None; + } + } } } + (Some(_), _) => unreachable!( + "Only rw-numeric[], rw_int256[] and varchar[] are supported to convert to pg-numeric[]" + ), + // This item is NULL, continue to handle next item. + (None, _) => None, + }; + builder.append(scalar); + } + Some(ScalarImpl::from(ListValue::new(builder.finish()))) + } + (ScalarAdapter::EnumList(vec), &DataType::List(dtype)) => { + let mut builder = dtype.create_array_builder(0); + for val in vec { + match val { + Some(EnumString(s)) => { + builder.append(Some(ScalarImpl::from(s))); + } + None => { + return None; + } } - _ => return None, - }, - (Some(_), _) => unreachable!( - "into_scalar_in_list should only be called with ScalarAdapter::Numeric types" - ), - (None, _) => None, - }; - builder.append(scalar); + } + Some(ScalarImpl::from(ListValue::new(builder.finish()))) + } + (ScalarAdapter::List(vec), &DataType::List(dtype)) => { + // Due to https://github.com/risingwavelabs/risingwave/issues/16882, INTERVAL_ARRAY is not supported in Debezium, so we keep backfilling and CDC consistent. + if matches!(**dtype, DataType::Interval) { + return None; + } + let mut builder = dtype.create_array_builder(0); + for val in vec { + builder.append(val.and_then(|v| v.into_scalar(dtype))); + } + Some(ScalarImpl::from(ListValue::new(builder.finish()))) + } + (scaler, ty) => { + tracing::error!( + adapter = scaler.name(), + rw_type = ty.pg_name(), + "failed to convert from ScalarAdapter: invalid conversion" + ); + None + } } - Some(builder) } } @@ -251,6 +332,23 @@ fn pg_numeric_to_rw_int256(val: &PgNumeric) -> Option { } } +fn pg_numeric_to_rw_numeric(val: &PgNumeric) -> Option { + match val { + PgNumeric::NegativeInf => Some(ScalarImpl::from(Decimal::NegativeInf)), + PgNumeric::Normalized(big_decimal) => { + match Decimal::from_str(big_decimal.to_string().as_str()) { + Ok(num) => Some(ScalarImpl::from(num)), + Err(err) => { + tracing::error!(error = %err.as_report(), "parse pg-numeric as rw-numeric failed (likely out-of-range"); + None + } + } + } + PgNumeric::PositiveInf => Some(ScalarImpl::from(Decimal::PositiveInf)), + PgNumeric::NaN => Some(ScalarImpl::from(Decimal::NaN)), + } +} + fn pg_numeric_to_string(val: &PgNumeric) -> String { // TODO(kexiang): NEGATIVE_INFINITY -> -Infinity, POSITIVE_INFINITY -> Infinity, NAN -> NaN // The current implementation is to ensure consistency with the behavior of cdc event parsor. @@ -262,11 +360,20 @@ fn pg_numeric_to_string(val: &PgNumeric) -> String { } } -fn string_to_pg_numeric(s: &str) -> super::ConnectorResult { - Ok(match s { +fn string_to_pg_numeric(s: &str) -> PgNumeric { + match s { "NEGATIVE_INFINITY" => PgNumeric::NegativeInf, "POSITIVE_INFINITY" => PgNumeric::PositiveInf, "NAN" => PgNumeric::NaN, _ => PgNumeric::Normalized(s.parse().unwrap()), - }) + } +} + +fn rw_numeric_to_pg_numeric(val: Decimal) -> PgNumeric { + match val { + Decimal::NegativeInf => PgNumeric::NegativeInf, + Decimal::Normalized(inner) => PgNumeric::Normalized(inner.to_string().parse().unwrap()), + Decimal::PositiveInf => PgNumeric::PositiveInf, + Decimal::NaN => PgNumeric::NaN, + } } diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index a4e3117b49519..4d4a89a3248cd 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -241,7 +241,7 @@ impl PostgresExternalTableReader { let stream = match start_pk_row { Some(ref pk_row) => { - let params: Vec>> = pk_row + let params: Vec> = pk_row .iter() .zip_eq_fast(self.prepared_scan_stmt.params()) .map(|(datum, ty)| { @@ -260,7 +260,7 @@ impl PostgresExternalTableReader { Self::get_normalized_table_name(&table_name), order_key, ); - let params: Vec>> = vec![]; + let params: Vec> = vec![]; client.query_raw(&sql, ¶ms).await? } }; From 11aac57ca2687e6b200402dccb577edbd86b5e85 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Wed, 29 May 2024 05:42:39 +0800 Subject: [PATCH 13/13] feat: create secret catalog (#16288) Signed-off-by: tabVersion Co-authored-by: August Co-authored-by: Eric Fu --- Cargo.lock | 40 ++++++- e2e_test/ddl/secret.slt | 23 ++++ proto/catalog.proto | 58 ++++++--- proto/ddl_service.proto | 22 ++++ proto/meta.proto | 2 + proto/secret.proto | 20 ++++ .../common_service/src/observer_manager.rs | 6 + src/common/src/catalog/mod.rs | 35 ++++++ src/common/src/config.rs | 27 ++++- src/config/docs.md | 1 + src/config/example.toml | 1 + src/connector/src/sink/catalog/desc.rs | 4 +- src/connector/src/sink/catalog/mod.rs | 5 + src/frontend/src/catalog/catalog_service.rs | 34 +++++- src/frontend/src/catalog/mod.rs | 3 + src/frontend/src/catalog/root_catalog.rs | 74 +++++++++++- src/frontend/src/catalog/schema_catalog.rs | 67 ++++++++++- src/frontend/src/catalog/secret_catalog.rs | 45 +++++++ src/frontend/src/handler/create_secret.rs | 102 ++++++++++++++++ src/frontend/src/handler/create_sink.rs | 4 +- src/frontend/src/handler/create_source.rs | 3 +- src/frontend/src/handler/drop_secret.rs | 64 ++++++++++ src/frontend/src/handler/mod.rs | 11 +- src/frontend/src/handler/show.rs | 6 + src/frontend/src/observer/observer_manager.rs | 17 ++- src/frontend/src/session.rs | 20 ++++ src/frontend/src/test_utils.rs | 17 ++- src/frontend/src/utils/with_options.rs | 9 ++ src/meta/Cargo.toml | 2 + src/meta/model_v2/migration/src/lib.rs | 2 + .../migration/src/m20240525_090457_secret.rs | 79 +++++++++++++ src/meta/model_v2/src/lib.rs | 4 + src/meta/model_v2/src/object.rs | 3 + src/meta/model_v2/src/prelude.rs | 1 + src/meta/model_v2/src/secret.rs | 57 +++++++++ src/meta/model_v2/src/sink.rs | 7 +- src/meta/node/src/lib.rs | 1 + src/meta/service/src/ddl_service.rs | 37 +++++- src/meta/service/src/notification_service.rs | 6 + src/meta/src/controller/catalog.rs | 111 ++++++++++++++++-- src/meta/src/controller/mod.rs | 26 +++- src/meta/src/controller/utils.rs | 28 ++++- src/meta/src/manager/catalog/database.rs | 48 +++++++- src/meta/src/manager/catalog/mod.rs | 57 ++++++++- src/meta/src/manager/env.rs | 4 + src/meta/src/manager/id.rs | 5 + src/meta/src/model/catalog.rs | 5 +- src/meta/src/model/mod.rs | 1 + src/meta/src/rpc/ddl_controller.rs | 110 +++++++++++++---- src/prost/build.rs | 1 + src/prost/src/lib.rs | 10 +- src/rpc_client/src/meta_client.rs | 31 ++++- src/sqlparser/src/ast/mod.rs | 39 ++++-- src/sqlparser/src/ast/statement.rs | 44 ++++++- src/sqlparser/src/keywords.rs | 2 + src/sqlparser/src/parser.rs | 18 ++- src/sqlparser/tests/testdata/create.yaml | 4 + src/sqlparser/tests/testdata/drop.yaml | 4 + src/sqlparser/tests/testdata/show.yaml | 6 + src/utils/pgwire/src/pg_response.rs | 5 + 60 files changed, 1387 insertions(+), 91 deletions(-) create mode 100644 e2e_test/ddl/secret.slt create mode 100644 proto/secret.proto create mode 100644 src/frontend/src/catalog/secret_catalog.rs create mode 100644 src/frontend/src/handler/create_secret.rs create mode 100644 src/frontend/src/handler/drop_secret.rs create mode 100644 src/meta/model_v2/migration/src/m20240525_090457_secret.rs create mode 100644 src/meta/model_v2/src/secret.rs diff --git a/Cargo.lock b/Cargo.lock index ab2f45a519bae..2cd740e2f31b2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -77,6 +77,22 @@ dependencies = [ "aes", ] +[[package]] +name = "aes-siv" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e08d0cdb774acd1e4dac11478b1a0c0d203134b2aab0ba25eb430de9b18f8b9" +dependencies = [ + "aead", + "aes", + "cipher", + "cmac", + "ctr", + "dbl", + "digest", + "zeroize", +] + [[package]] name = "ahash" version = "0.7.8" @@ -2526,6 +2542,17 @@ dependencies = [ "cc", ] +[[package]] +name = "cmac" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8543454e3c3f5126effff9cd44d562af4e31fb8ce1cc0d3dcd8f084515dbc1aa" +dependencies = [ + "cipher", + "dbl", + "digest", +] + [[package]] name = "cmake" version = "0.1.50" @@ -3604,6 +3631,15 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "dbl" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd2735a791158376708f9347fe8faba9667589d82427ef3aed6794a8981de3d9" +dependencies = [ + "generic-array", +] + [[package]] name = "debugid" version = "0.8.0" @@ -9374,7 +9410,7 @@ dependencies = [ "indoc", "libc", "memoffset", - "parking_lot 0.11.2", + "parking_lot 0.12.1", "portable-atomic", "pyo3-build-config", "pyo3-ffi", @@ -11049,6 +11085,7 @@ dependencies = [ name = "risingwave_meta" version = "1.9.0-alpha" dependencies = [ + "aes-siv", "anyhow", "arc-swap", "assert_matches", @@ -11056,6 +11093,7 @@ dependencies = [ "aws-config", "axum 0.7.4", "base64-url", + "bincode 1.3.3", "bytes", "chrono", "clap", diff --git a/e2e_test/ddl/secret.slt b/e2e_test/ddl/secret.slt new file mode 100644 index 0000000000000..7c11e2e6245a3 --- /dev/null +++ b/e2e_test/ddl/secret.slt @@ -0,0 +1,23 @@ +statement error secret backend "fake-backend" is not supported +create secret secret_1 with ( + backend = 'fake-backend' +) as 'demo_secret'; + +statement ok +create secret secret_1 with ( + backend = 'meta' +) as 'demo_secret'; + +# wait for support for hashicorp_vault backend +# statement ok +# create secret secret_2 with ( +# backend = 'hashicorp_vault' +# ); + +query T +show secrets; +---- +secret_1 + +statement ok +drop secret secret_1; diff --git a/proto/catalog.proto b/proto/catalog.proto index 42b672a1b6c51..7dfefa003217d 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -62,10 +62,12 @@ message StreamSourceInfo { SchemaRegistryNameStrategy name_strategy = 10; optional string key_message_name = 11; plan_common.ExternalTableDesc external_table = 12; - // **This field should now be called `is_shared`.** Not renamed for backwards compatibility. + // **This field should now be called `is_shared`.** Not renamed for backwards + // compatibility. // // Whether the stream source is a shared source (it has a streaming job). - // This is related with [RFC: Reusable Source Executor](https://github.com/risingwavelabs/rfcs/pull/72). + // This is related with [RFC: Reusable Source + // Executor](https://github.com/risingwavelabs/rfcs/pull/72). // // Currently, the following sources can be shared: // @@ -80,6 +82,9 @@ message StreamSourceInfo { bool is_distributed = 15; // Options specified by user in the FORMAT ENCODE clause. map format_encode_options = 14; + + // Handle the source relies on any sceret. The key is the propertity name and the value is the secret id. + map secret_ref = 16; } message Source { @@ -174,6 +179,9 @@ message Sink { // Whether it should use background ddl or block until backfill finishes. CreateType create_type = 24; + + // Handle the sink relies on any sceret. The key is the propertity name and the value is the secret id. + map secret_ref = 25; } message Subscription { @@ -239,7 +247,8 @@ message Index { optional uint64 created_at_epoch = 11; StreamJobStatus stream_job_status = 12; - // Use to record the prefix len of the index_item to reconstruct index columns provided by users. + // Use to record the prefix len of the index_item to reconstruct index columns + // provided by users. uint32 index_columns_len = 13; // Cluster version (tracked by git commit) when initialized/created optional string initialized_at_cluster_version = 14; @@ -319,8 +328,8 @@ message Table { // an optional column index which is the vnode of each row computed by the // table's consistent hash distribution optional uint32 vnode_col_index = 18; - // An optional column index of row id. If the primary key is specified by users, - // this will be `None`. + // An optional column index of row id. If the primary key is specified by + // users, this will be `None`. optional uint32 row_id_index = 19; // The column indices which are stored in the state store's value with // row-encoding. Currently is not supported yet and expected to be @@ -329,23 +338,26 @@ message Table { string definition = 21; // Used to control whether handling pk conflict for incoming data. HandleConflictBehavior handle_pk_conflict_behavior = 22; - // Anticipated read prefix pattern (number of fields) for the table, which can be utilized - // for implementing the table's bloom filter or other storage optimization techniques. + // Anticipated read prefix pattern (number of fields) for the table, which can + // be utilized for implementing the table's bloom filter or other storage + // optimization techniques. uint32 read_prefix_len_hint = 23; repeated int32 watermark_indices = 24; repeated int32 dist_key_in_pk = 25; - // A dml fragment id corresponds to the table, used to decide where the dml statement is executed. + // A dml fragment id corresponds to the table, used to decide where the dml + // statement is executed. optional uint32 dml_fragment_id = 26; // The range of row count of the table. - // This field is not always present due to backward compatibility. Use `Cardinality::unknown` in this case. + // This field is not always present due to backward compatibility. Use + // `Cardinality::unknown` in this case. plan_common.Cardinality cardinality = 27; optional uint64 initialized_at_epoch = 28; optional uint64 created_at_epoch = 29; - // This field is introduced in v1.2.0. It is used to indicate whether the table should use - // watermark_cache to avoid state cleaning as a performance optimization. - // In older versions we can just initialize without it. + // This field is introduced in v1.2.0. It is used to indicate whether the + // table should use watermark_cache to avoid state cleaning as a performance + // optimization. In older versions we can just initialize without it. bool cleaned_by_watermark = 30; // Used to filter created / creating tables in meta. @@ -363,14 +375,18 @@ message Table { optional string initialized_at_cluster_version = 35; optional string created_at_cluster_version = 36; - // TTL of the record in the table, to ensure the consistency with other tables in the streaming plan, it only applies to append-only tables. + // TTL of the record in the table, to ensure the consistency with other tables + // in the streaming plan, it only applies to append-only tables. optional uint32 retention_seconds = 37; - // This field specifies the index of the column set in the "with version column" within all the columns. It is used for filtering during "on conflict" operations. + // This field specifies the index of the column set in the "with version + // column" within all the columns. It is used for filtering during "on + // conflict" operations. optional uint32 version_column_index = 38; - // Per-table catalog version, used by schema change. `None` for internal tables and tests. - // Not to be confused with the global catalog version for notification service. + // Per-table catalog version, used by schema change. `None` for internal + // tables and tests. Not to be confused with the global catalog version for + // notification service. TableVersion version = 100; } @@ -415,3 +431,13 @@ message Comment { optional uint32 column_index = 4; optional string description = 5; } + +message Secret { + uint32 id = 1; + string name = 2; + uint32 database_id = 3; + // The secret here is encrypted to bytes. + bytes value = 4; + uint32 owner = 5; + uint32 schema_id = 6; +} diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 58fb645e056eb..46c2a5c22ff6d 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -357,6 +357,26 @@ message GetDdlProgressResponse { repeated DdlProgress ddl_progress = 1; } +message CreateSecretRequest { + string name = 1; + bytes value = 2; + uint32 database_id = 3; + uint32 schema_id = 4; + uint32 owner_id = 5; +} + +message CreateSecretResponse { + uint64 version = 1; +} + +message DropSecretRequest { + uint32 secret_id = 1; +} + +message DropSecretResponse { + uint64 version = 1; +} + message CreateConnectionRequest { message PrivateLink { catalog.Connection.PrivateLinkService.PrivateLinkProvider provider = 1; @@ -427,6 +447,8 @@ service DdlService { rpc CreateMaterializedView(CreateMaterializedViewRequest) returns (CreateMaterializedViewResponse); rpc DropMaterializedView(DropMaterializedViewRequest) returns (DropMaterializedViewResponse); rpc CreateTable(CreateTableRequest) returns (CreateTableResponse); + rpc CreateSecret(CreateSecretRequest) returns (CreateSecretResponse); + rpc DropSecret(DropSecretRequest) returns (DropSecretResponse); rpc AlterName(AlterNameRequest) returns (AlterNameResponse); rpc AlterSource(AlterSourceRequest) returns (AlterSourceResponse); rpc AlterOwner(AlterOwnerRequest) returns (AlterOwnerResponse); diff --git a/proto/meta.proto b/proto/meta.proto index df90d99f6e9e9..9ad18cb3df7d5 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -407,6 +407,7 @@ message MetaSnapshot { reserved 9; reserved "parallel_unit_mappings"; GetSessionParamsResponse session_params = 20; + repeated catalog.Secret secrets = 23; repeated common.WorkerNode nodes = 10; hummock.HummockSnapshot hummock_snapshot = 11; hummock.HummockVersion hummock_version = 12; @@ -469,6 +470,7 @@ message SubscribeResponse { Recovery recovery = 25; FragmentWorkerSlotMapping streaming_worker_slot_mapping = 27; FragmentWorkerSlotMappings serving_worker_slot_mappings = 28; + catalog.Secret secret = 29; } reserved 12; reserved "parallel_unit_mapping"; diff --git a/proto/secret.proto b/proto/secret.proto new file mode 100644 index 0000000000000..f5065009519fd --- /dev/null +++ b/proto/secret.proto @@ -0,0 +1,20 @@ +syntax = "proto3"; + +package secret; + +message SecretMetaBackend { + bytes value = 1; +} + +message SecretHashicropValutBackend { + string host = 1; + string vault_token = 2; +} + +message Secret { + // the message is stored in meta as encrypted bytes and is interpreted as bytes by catalog + oneof secret_backend { + SecretMetaBackend meta = 1; + SecretHashicropValutBackend hashicorp_vault = 2; + } +} diff --git a/src/common/common_service/src/observer_manager.rs b/src/common/common_service/src/observer_manager.rs index a611c40aebc68..e760a0e16866c 100644 --- a/src/common/common_service/src/observer_manager.rs +++ b/src/common/common_service/src/observer_manager.rs @@ -27,6 +27,7 @@ pub trait SubscribeTypeEnum { } pub struct SubscribeFrontend {} + impl SubscribeTypeEnum for SubscribeFrontend { fn subscribe_type() -> SubscribeType { SubscribeType::Frontend @@ -34,6 +35,7 @@ impl SubscribeTypeEnum for SubscribeFrontend { } pub struct SubscribeHummock {} + impl SubscribeTypeEnum for SubscribeHummock { fn subscribe_type() -> SubscribeType { SubscribeType::Hummock @@ -41,6 +43,7 @@ impl SubscribeTypeEnum for SubscribeHummock { } pub struct SubscribeCompactor {} + impl SubscribeTypeEnum for SubscribeCompactor { fn subscribe_type() -> SubscribeType { SubscribeType::Compactor @@ -48,6 +51,7 @@ impl SubscribeTypeEnum for SubscribeCompactor { } pub struct SubscribeCompute {} + impl SubscribeTypeEnum for SubscribeCompute { fn subscribe_type() -> SubscribeType { SubscribeType::Compute @@ -142,6 +146,7 @@ where | Info::RelationGroup(_) | Info::User(_) | Info::Connection(_) + | Info::Secret(_) | Info::Function(_) => { notification.version > info.version.as_ref().unwrap().catalog_version } @@ -231,6 +236,7 @@ where } } } + const RE_SUBSCRIBE_RETRY_INTERVAL: Duration = Duration::from_millis(100); #[async_trait::async_trait] diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index cef5b021b9dbd..86c6e8895c066 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -451,6 +451,41 @@ impl From for u32 { } } +#[derive(Clone, Copy, Debug, Display, Default, Hash, PartialOrd, PartialEq, Eq, Ord)] +pub struct SecretId(pub u32); + +impl SecretId { + pub const fn new(id: u32) -> Self { + SecretId(id) + } + + pub const fn placeholder() -> Self { + SecretId(OBJECT_ID_PLACEHOLDER) + } + + pub fn secret_id(&self) -> u32 { + self.0 + } +} + +impl From for SecretId { + fn from(id: u32) -> Self { + Self::new(id) + } +} + +impl From<&u32> for SecretId { + fn from(id: &u32) -> Self { + Self::new(*id) + } +} + +impl From for u32 { + fn from(id: SecretId) -> Self { + id.0 + } +} + #[derive(Default, Debug, Clone, Copy, PartialEq, Eq, Hash)] pub enum ConflictBehavior { #[default] diff --git a/src/common/src/config.rs b/src/common/src/config.rs index cd276e8a966f4..8ae14702d3261 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -366,6 +366,9 @@ pub struct MetaConfig { /// Whether compactor should rewrite row to remove dropped column. #[serde(default = "default::meta::enable_dropped_column_reclaim")] pub enable_dropped_column_reclaim: bool, + + #[serde(default = "default::meta::secret_store_private_key")] + pub secret_store_private_key: Vec, } #[derive(Copy, Clone, Debug, Default)] @@ -1322,9 +1325,14 @@ pub mod default { pub fn parallelism_control_trigger_first_delay_sec() -> u64 { 30 } + pub fn enable_dropped_column_reclaim() -> bool { false } + + pub fn secret_store_private_key() -> Vec { + "demo-secret-private-key".as_bytes().to_vec() + } } pub mod server { @@ -1492,6 +1500,7 @@ pub mod default { pub fn max_preload_io_retry_times() -> usize { 3 } + pub fn mem_table_spill_threshold() -> usize { 4 << 20 } @@ -1532,7 +1541,6 @@ pub mod default { } pub mod file_cache { - pub fn dir() -> String { "".to_string() } @@ -1786,7 +1794,8 @@ pub mod default { // decrease this configure when the generation of checkpoint barrier is not frequent. const DEFAULT_TIER_COMPACT_TRIGGER_NUMBER: u64 = 12; - const DEFAULT_TARGET_FILE_SIZE_BASE: u64 = 32 * 1024 * 1024; // 32MB + const DEFAULT_TARGET_FILE_SIZE_BASE: u64 = 32 * 1024 * 1024; + // 32MB const DEFAULT_MAX_SUB_COMPACTION: u32 = 4; const DEFAULT_LEVEL_MULTIPLIER: u64 = 5; const DEFAULT_MAX_SPACE_RECLAIM_BYTES: u64 = 512 * 1024 * 1024; // 512MB; @@ -1802,42 +1811,55 @@ pub mod default { pub fn max_bytes_for_level_base() -> u64 { DEFAULT_MAX_BYTES_FOR_LEVEL_BASE } + pub fn max_bytes_for_level_multiplier() -> u64 { DEFAULT_LEVEL_MULTIPLIER } + pub fn max_compaction_bytes() -> u64 { DEFAULT_MAX_COMPACTION_BYTES } + pub fn sub_level_max_compaction_bytes() -> u64 { DEFAULT_MIN_COMPACTION_BYTES } + pub fn level0_tier_compact_file_number() -> u64 { DEFAULT_TIER_COMPACT_TRIGGER_NUMBER } + pub fn target_file_size_base() -> u64 { DEFAULT_TARGET_FILE_SIZE_BASE } + pub fn compaction_filter_mask() -> u32 { (CompactionFilterFlag::STATE_CLEAN | CompactionFilterFlag::TTL).into() } + pub fn max_sub_compaction() -> u32 { DEFAULT_MAX_SUB_COMPACTION } + pub fn level0_stop_write_threshold_sub_level_number() -> u64 { DEFAULT_LEVEL0_STOP_WRITE_THRESHOLD_SUB_LEVEL_NUMBER } + pub fn level0_sub_level_compact_level_count() -> u32 { DEFAULT_MIN_SUB_LEVEL_COMPACT_LEVEL_COUNT } + pub fn level0_overlapping_sub_level_compact_level_count() -> u32 { DEFAULT_MIN_OVERLAPPING_SUB_LEVEL_COMPACT_LEVEL_COUNT } + pub fn max_space_reclaim_bytes() -> u64 { DEFAULT_MAX_SPACE_RECLAIM_BYTES } + pub fn level0_max_compact_file_number() -> u64 { DEFAULT_MAX_COMPACTION_FILE_COUNT } + pub fn tombstone_reclaim_ratio() -> u32 { DEFAULT_TOMBSTONE_RATIO_PERCENT } @@ -1962,6 +1984,7 @@ pub mod default { pub mod developer { use crate::util::env_var::env_var_is_true_or; + const RW_USE_OPENDAL_FOR_S3: &str = "RW_USE_OPENDAL_FOR_S3"; pub fn object_store_retry_unknown_service_error() -> bool { diff --git a/src/config/docs.md b/src/config/docs.md index aea210f5235af..2f8c4ce2812b1 100644 --- a/src/config/docs.md +++ b/src/config/docs.md @@ -56,6 +56,7 @@ This page is automatically generated by `./risedev generate-example-config` | periodic_split_compact_group_interval_sec | | 10 | | periodic_tombstone_reclaim_compaction_interval_sec | | 600 | | periodic_ttl_reclaim_compaction_interval_sec | Schedule `ttl_reclaim` compaction for all compaction groups with this interval. | 1800 | +| secret_store_private_key | | [100, 101, 109, 111, 45, 115, 101, 99, 114, 101, 116, 45, 112, 114, 105, 118, 97, 116, 101, 45, 107, 101, 121] | | split_group_size_limit | | 68719476736 | | table_write_throughput_threshold | The threshold of write throughput to trigger a group split. Increase this configuration value to avoid split too many groups with few data write. | 16777216 | | unrecognized | | | diff --git a/src/config/example.toml b/src/config/example.toml index a50b4b8c10d65..fb2243535d6a4 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -54,6 +54,7 @@ compact_task_table_size_partition_threshold_high = 536870912 event_log_enabled = true event_log_channel_max_size = 10 enable_dropped_column_reclaim = false +secret_store_private_key = [100, 101, 109, 111, 45, 115, 101, 99, 114, 101, 116, 45, 112, 114, 105, 118, 97, 116, 101, 45, 107, 101, 121] [meta.compaction_config] max_bytes_for_level_base = 536870912 diff --git a/src/connector/src/sink/catalog/desc.rs b/src/connector/src/sink/catalog/desc.rs index 0fb466e5a1742..e5415c268d569 100644 --- a/src/connector/src/sink/catalog/desc.rs +++ b/src/connector/src/sink/catalog/desc.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use itertools::Itertools; use risingwave_common::catalog::{ @@ -83,6 +83,7 @@ impl SinkDesc { owner: UserId, connection_id: Option, dependent_relations: Vec, + secret_ref: HashMap, ) -> SinkCatalog { SinkCatalog { id: self.id, @@ -108,6 +109,7 @@ impl SinkDesc { created_at_cluster_version: None, initialized_at_cluster_version: None, create_type: self.create_type, + secret_ref, } } diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index 7eaa977e16044..bf5dd89dd7894 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -337,6 +337,9 @@ pub struct SinkCatalog { pub created_at_cluster_version: Option, pub initialized_at_cluster_version: Option, pub create_type: CreateType, + + /// The secret reference for the sink, mapping from property name to secret id. + pub secret_ref: HashMap, } impl SinkCatalog { @@ -378,6 +381,7 @@ impl SinkCatalog { created_at_cluster_version: self.created_at_cluster_version.clone(), initialized_at_cluster_version: self.initialized_at_cluster_version.clone(), create_type: self.create_type.to_proto() as i32, + secret_ref: self.secret_ref.clone(), } } @@ -471,6 +475,7 @@ impl From for SinkCatalog { initialized_at_cluster_version: pb.initialized_at_cluster_version, created_at_cluster_version: pb.created_at_cluster_version, create_type: CreateType::from_proto(create_type), + secret_ref: pb.secret_ref, } } } diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index b97c551f2ef78..f2bcdd2b62e12 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -34,7 +34,7 @@ use risingwave_rpc_client::MetaClient; use tokio::sync::watch::Receiver; use super::root_catalog::Catalog; -use super::{DatabaseId, TableId}; +use super::{DatabaseId, SecretId, TableId}; use crate::error::Result; use crate::user::UserId; @@ -43,6 +43,7 @@ pub type CatalogReadGuard = ArcRwLockReadGuard; /// [`CatalogReader`] can read catalog from local catalog and force the holder can not modify it. #[derive(Clone)] pub struct CatalogReader(Arc>); + impl CatalogReader { pub fn new(inner: Arc>) -> Self { CatalogReader(inner) @@ -130,6 +131,15 @@ pub trait CatalogWriter: Send + Sync { connection: create_connection_request::Payload, ) -> Result<()>; + async fn create_secret( + &self, + secret_name: String, + database_id: u32, + schema_id: u32, + owner_id: u32, + payload: Vec, + ) -> Result<()>; + async fn comment_on(&self, comment: PbComment) -> Result<()>; async fn drop_table( @@ -164,6 +174,8 @@ pub trait CatalogWriter: Send + Sync { async fn drop_connection(&self, connection_id: u32) -> Result<()>; + async fn drop_secret(&self, secret_id: SecretId) -> Result<()>; + async fn alter_table_name(&self, table_id: u32, table_name: &str) -> Result<()>; async fn alter_view_name(&self, view_id: u32, view_name: &str) -> Result<()>; @@ -373,6 +385,21 @@ impl CatalogWriter for CatalogWriterImpl { self.wait_version(version).await } + async fn create_secret( + &self, + secret_name: String, + database_id: u32, + schema_id: u32, + owner_id: u32, + payload: Vec, + ) -> Result<()> { + let version = self + .meta_client + .create_secret(secret_name, database_id, schema_id, owner_id, payload) + .await?; + self.wait_version(version).await + } + async fn comment_on(&self, comment: PbComment) -> Result<()> { let version = self.meta_client.comment_on(comment).await?; self.wait_version(version).await @@ -455,6 +482,11 @@ impl CatalogWriter for CatalogWriterImpl { self.wait_version(version).await } + async fn drop_secret(&self, secret_id: SecretId) -> Result<()> { + let version = self.meta_client.drop_secret(secret_id).await?; + self.wait_version(version).await + } + async fn alter_table_name(&self, table_id: u32, table_name: &str) -> Result<()> { let version = self .meta_client diff --git a/src/frontend/src/catalog/mod.rs b/src/frontend/src/catalog/mod.rs index 687fa5ac350e7..64c3f525dd8f6 100644 --- a/src/frontend/src/catalog/mod.rs +++ b/src/frontend/src/catalog/mod.rs @@ -39,6 +39,8 @@ pub(crate) mod system_catalog; pub(crate) mod table_catalog; pub(crate) mod view_catalog; +pub(crate) mod secret_catalog; + pub(crate) use catalog_service::CatalogReader; pub use index_catalog::IndexCatalog; pub use table_catalog::TableCatalog; @@ -55,6 +57,7 @@ pub(crate) type SchemaId = u32; pub(crate) type TableId = risingwave_common::catalog::TableId; pub(crate) type ColumnId = risingwave_common::catalog::ColumnId; pub(crate) type FragmentId = u32; +pub(crate) type SecretId = risingwave_common::catalog::SecretId; /// Check if the column name does not conflict with the internally reserved column name. pub fn check_valid_column_name(column_name: &str) -> Result<()> { diff --git a/src/frontend/src/catalog/root_catalog.rs b/src/frontend/src/catalog/root_catalog.rs index 7a99c199446dd..5e6431065ce79 100644 --- a/src/frontend/src/catalog/root_catalog.rs +++ b/src/frontend/src/catalog/root_catalog.rs @@ -21,8 +21,8 @@ use risingwave_common::session_config::{SearchPath, USER_NAME_WILD_CARD}; use risingwave_common::types::DataType; use risingwave_connector::sink::catalog::SinkCatalog; use risingwave_pb::catalog::{ - PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbSubscription, - PbTable, PbView, + PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSecret, PbSink, PbSource, + PbSubscription, PbTable, PbView, }; use risingwave_pb::hummock::HummockVersionStats; @@ -30,10 +30,13 @@ use super::function_catalog::FunctionCatalog; use super::source_catalog::SourceCatalog; use super::subscription_catalog::SubscriptionCatalog; use super::view_catalog::ViewCatalog; -use super::{CatalogError, CatalogResult, ConnectionId, SinkId, SourceId, SubscriptionId, ViewId}; +use super::{ + CatalogError, CatalogResult, ConnectionId, SecretId, SinkId, SourceId, SubscriptionId, ViewId, +}; use crate::catalog::connection_catalog::ConnectionCatalog; use crate::catalog::database_catalog::DatabaseCatalog; use crate::catalog::schema_catalog::SchemaCatalog; +use crate::catalog::secret_catalog::SecretCatalog; use crate::catalog::system_catalog::{ get_sys_tables_in_schema, get_sys_views_in_schema, SystemTableCatalog, }; @@ -201,6 +204,14 @@ impl Catalog { .create_subscription(proto); } + pub fn create_secret(&mut self, proto: &PbSecret) { + self.get_database_mut(proto.database_id) + .unwrap() + .get_schema_mut(proto.schema_id) + .unwrap() + .create_secret(proto); + } + pub fn create_view(&mut self, proto: &PbView) { self.get_database_mut(proto.database_id) .unwrap() @@ -257,6 +268,25 @@ impl Catalog { } } + pub fn update_secret(&mut self, proto: &PbSecret) { + let database = self.get_database_mut(proto.database_id).unwrap(); + let schema = database.get_schema_mut(proto.schema_id).unwrap(); + let secret_id = SecretId::new(proto.id); + if schema.get_secret_by_id(&secret_id).is_some() { + schema.update_secret(proto); + } else { + // Enter this branch when schema is changed by `ALTER ... SET SCHEMA ...` statement. + schema.create_secret(proto); + database + .iter_schemas_mut() + .find(|schema| { + schema.id() != proto.schema_id && schema.get_secret_by_id(&secret_id).is_some() + }) + .unwrap() + .drop_secret(secret_id); + } + } + pub fn drop_database(&mut self, db_id: DatabaseId) { let name = self.db_name_by_id.remove(&db_id).unwrap(); let database = self.database_by_name.remove(&name).unwrap(); @@ -377,6 +407,14 @@ impl Catalog { .drop_sink(sink_id); } + pub fn drop_secret(&mut self, db_id: DatabaseId, schema_id: SchemaId, secret_id: SecretId) { + self.get_database_mut(db_id) + .unwrap() + .get_schema_mut(schema_id) + .unwrap() + .drop_secret(secret_id); + } + pub fn update_sink(&mut self, proto: &PbSink) { let database = self.get_database_mut(proto.database_id).unwrap(); let schema = database.get_schema_mut(proto.schema_id).unwrap(); @@ -793,6 +831,21 @@ impl Catalog { Err(CatalogError::NotFound("view", view_id.to_string())) } + pub fn get_secret_by_name<'a>( + &self, + db_name: &str, + schema_path: SchemaPath<'a>, + secret_name: &str, + ) -> CatalogResult<(&Arc, &'a str)> { + schema_path + .try_find(|schema_name| { + Ok(self + .get_schema_by_name(db_name, schema_name)? + .get_secret_by_name(secret_name)) + })? + .ok_or_else(|| CatalogError::NotFound("secret", secret_name.to_string())) + } + pub fn get_connection_by_name<'a>( &self, db_name: &str, @@ -958,6 +1011,21 @@ impl Catalog { } } + pub fn check_secret_name_duplicated( + &self, + db_name: &str, + schema_name: &str, + secret_name: &str, + ) -> CatalogResult<()> { + let schema = self.get_schema_by_name(db_name, schema_name)?; + + if schema.get_secret_by_name(secret_name).is_some() { + Err(CatalogError::Duplicated("secret", secret_name.to_string())) + } else { + Ok(()) + } + } + /// Get the catalog cache's catalog version. pub fn version(&self) -> u64 { self.version diff --git a/src/frontend/src/catalog/schema_catalog.rs b/src/frontend/src/catalog/schema_catalog.rs index 20a99ad820af8..fffb171c4c8bc 100644 --- a/src/frontend/src/catalog/schema_catalog.rs +++ b/src/frontend/src/catalog/schema_catalog.rs @@ -22,7 +22,8 @@ use risingwave_common::types::DataType; use risingwave_connector::sink::catalog::SinkCatalog; pub use risingwave_expr::sig::*; use risingwave_pb::catalog::{ - PbConnection, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbSubscription, PbTable, PbView, + PbConnection, PbFunction, PbIndex, PbSchema, PbSecret, PbSink, PbSource, PbSubscription, + PbTable, PbView, }; use risingwave_pb::user::grant_privilege::Object; @@ -31,11 +32,12 @@ use super::{OwnedByUserCatalog, SubscriptionId}; use crate::catalog::connection_catalog::ConnectionCatalog; use crate::catalog::function_catalog::FunctionCatalog; use crate::catalog::index_catalog::IndexCatalog; +use crate::catalog::secret_catalog::SecretCatalog; use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::system_catalog::SystemTableCatalog; use crate::catalog::table_catalog::TableCatalog; use crate::catalog::view_catalog::ViewCatalog; -use crate::catalog::{ConnectionId, DatabaseId, SchemaId, SinkId, SourceId, ViewId}; +use crate::catalog::{ConnectionId, DatabaseId, SchemaId, SecretId, SinkId, SourceId, ViewId}; use crate::expr::{infer_type_name, infer_type_with_sigmap, Expr, ExprImpl}; use crate::user::UserId; @@ -62,6 +64,11 @@ pub struct SchemaCatalog { function_by_id: HashMap>, connection_by_name: HashMap>, connection_by_id: HashMap>, + secret_by_name: HashMap>, + secret_by_id: HashMap>, + + _secret_source_ref: HashMap>, + _secret_sink_ref: HashMap>, // This field is currently used only for `show connections` connection_source_ref: HashMap>, @@ -484,6 +491,46 @@ impl SchemaCatalog { .expect("connection not found by name"); } + pub fn create_secret(&mut self, prost: &PbSecret) { + let name = prost.name.clone(); + let id = SecretId::new(prost.id); + let secret = SecretCatalog::from(prost); + let secret_ref = Arc::new(secret); + + self.secret_by_id + .try_insert(id, secret_ref.clone()) + .unwrap(); + self.secret_by_name + .try_insert(name, secret_ref.clone()) + .unwrap(); + } + + pub fn update_secret(&mut self, prost: &PbSecret) { + let name = prost.name.clone(); + let id = SecretId::new(prost.id); + let secret = SecretCatalog::from(prost); + let secret_ref = Arc::new(secret); + + let old_secret = self.secret_by_id.get(&id).unwrap(); + // check if secret name get updated. + if old_secret.name != name { + self.secret_by_name.remove(&old_secret.name); + } + + self.secret_by_name.insert(name, secret_ref.clone()); + self.secret_by_id.insert(id, secret_ref); + } + + pub fn drop_secret(&mut self, secret_id: SecretId) { + let secret_ref = self + .secret_by_id + .remove(&secret_id) + .expect("secret not found by id"); + self.secret_by_name + .remove(&secret_ref.name) + .expect("secret not found by name"); + } + pub fn iter_all(&self) -> impl Iterator> { self.table_by_name.values() } @@ -546,6 +593,10 @@ impl SchemaCatalog { self.connection_by_name.values() } + pub fn iter_secret(&self) -> impl Iterator> { + self.secret_by_name.values() + } + pub fn iter_system_tables(&self) -> impl Iterator> { self.system_table_by_name.values() } @@ -687,6 +738,14 @@ impl SchemaCatalog { self.connection_by_name.get(connection_name) } + pub fn get_secret_by_name(&self, secret_name: &str) -> Option<&Arc> { + self.secret_by_name.get(secret_name) + } + + pub fn get_secret_by_id(&self, secret_id: &SecretId) -> Option<&Arc> { + self.secret_by_id.get(secret_id) + } + /// get all sources referencing the connection pub fn get_source_ids_by_connection( &self, @@ -764,6 +823,10 @@ impl From<&PbSchema> for SchemaCatalog { function_by_id: HashMap::new(), connection_by_name: HashMap::new(), connection_by_id: HashMap::new(), + secret_by_name: HashMap::new(), + secret_by_id: HashMap::new(), + _secret_source_ref: HashMap::new(), + _secret_sink_ref: HashMap::new(), connection_source_ref: HashMap::new(), connection_sink_ref: HashMap::new(), subscription_by_name: HashMap::new(), diff --git a/src/frontend/src/catalog/secret_catalog.rs b/src/frontend/src/catalog/secret_catalog.rs new file mode 100644 index 0000000000000..5e9aaae7dec99 --- /dev/null +++ b/src/frontend/src/catalog/secret_catalog.rs @@ -0,0 +1,45 @@ +// Copyright 2024 RisingWave Labs +// +// 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 risingwave_pb::catalog::PbSecret; + +use crate::catalog::{DatabaseId, OwnedByUserCatalog, SecretId}; +use crate::user::UserId; + +#[derive(Clone, Debug, PartialEq, Eq, Hash)] +pub struct SecretCatalog { + pub secret_id: SecretId, + pub name: String, + pub database_id: DatabaseId, + pub value: Vec, + pub owner: UserId, +} + +impl From<&PbSecret> for SecretCatalog { + fn from(value: &PbSecret) -> Self { + Self { + secret_id: SecretId::new(value.id), + database_id: value.database_id, + owner: value.owner, + name: value.name.clone(), + value: value.value.clone(), + } + } +} + +impl OwnedByUserCatalog for SecretCatalog { + fn owner(&self) -> UserId { + self.owner + } +} diff --git a/src/frontend/src/handler/create_secret.rs b/src/frontend/src/handler/create_secret.rs new file mode 100644 index 0000000000000..2e99f26e97cb8 --- /dev/null +++ b/src/frontend/src/handler/create_secret.rs @@ -0,0 +1,102 @@ +// Copyright 2024 RisingWave Labs +// +// 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 pgwire::pg_response::{PgResponse, StatementType}; +use prost::Message; +use risingwave_common::bail_not_implemented; +use risingwave_sqlparser::ast::{CreateSecretStatement, SqlOption, Value}; + +use crate::error::{ErrorCode, Result}; +use crate::handler::{HandlerArgs, RwPgResponse}; +use crate::{Binder, WithOptions}; + +const SECRET_BACKEND_KEY: &str = "backend"; + +const SECRET_BACKEND_META: &str = "meta"; +const SECRET_BACKEND_HASHICORP_VAULT: &str = "hashicorp_vault"; + +pub async fn handle_create_secret( + handler_args: HandlerArgs, + stmt: CreateSecretStatement, +) -> Result { + let session = handler_args.session.clone(); + let db_name = session.database(); + let (schema_name, connection_name) = + Binder::resolve_schema_qualified_name(db_name, stmt.secret_name.clone())?; + + if let Err(e) = session.check_secret_name_duplicated(stmt.secret_name.clone()) { + return if stmt.if_not_exists { + Ok(PgResponse::builder(StatementType::CREATE_SECRET) + .notice(format!("secret \"{}\" exists, skipping", connection_name)) + .into()) + } else { + Err(e) + }; + } + + // check if the secret backend is supported + let with_props = WithOptions::try_from(stmt.with_properties.0.as_ref() as &[SqlOption])?; + let secret_payload: Vec = { + if let Some(backend) = with_props.inner().get(SECRET_BACKEND_KEY) { + match backend.to_lowercase().as_ref() { + SECRET_BACKEND_META => { + let backend = risingwave_pb::secret::Secret { + secret_backend: Some(risingwave_pb::secret::secret::SecretBackend::Meta( + risingwave_pb::secret::SecretMetaBackend { value: vec![] }, + )), + }; + backend.encode_to_vec() + } + SECRET_BACKEND_HASHICORP_VAULT => { + if stmt.credential != Value::Null { + return Err(ErrorCode::InvalidParameterValue( + "credential must be null for hashicorp_vault backend".to_string(), + ) + .into()); + } + bail_not_implemented!("hashicorp_vault backend is not implemented yet") + } + _ => { + return Err(ErrorCode::InvalidParameterValue(format!( + "secret backend \"{}\" is not supported. Supported backends are: {}", + backend, + [SECRET_BACKEND_META, SECRET_BACKEND_HASHICORP_VAULT].join(",") + )) + .into()); + } + } + } else { + return Err(ErrorCode::InvalidParameterValue(format!( + "secret backend is not specified in with clause. Supported backends are: {}", + [SECRET_BACKEND_META, SECRET_BACKEND_HASHICORP_VAULT].join(",") + )) + .into()); + } + }; + + let (database_id, schema_id) = session.get_database_and_schema_id_for_create(schema_name)?; + + let catalog_writer = session.catalog_writer()?; + catalog_writer + .create_secret( + stmt.secret_name.real_value(), + database_id, + schema_id, + session.user_id(), + secret_payload, + ) + .await?; + + Ok(PgResponse::empty_result(StatementType::CREATE_SECRET)) +} diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index b81a40f6d5759..6f82425fa06e8 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -63,7 +63,7 @@ use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, RelationC use crate::scheduler::streaming_manager::CreatingStreamingJobInfo; use crate::session::SessionImpl; use crate::stream_fragmenter::build_graph; -use crate::utils::resolve_privatelink_in_with_option; +use crate::utils::{resolve_privatelink_in_with_option, resolve_secret_in_with_options}; use crate::{Explain, Planner, TableCatalog, WithOptions}; // used to store result of `gen_sink_plan` @@ -145,6 +145,7 @@ pub fn gen_sink_plan( resolve_privatelink_in_with_option(&mut with_options, &sink_schema_name, session)?; conn_id.map(ConnectionId) }; + let secret_ref = resolve_secret_in_with_options(&mut with_options, session)?; let emit_on_window_close = stmt.emit_mode == Some(EmitMode::OnWindowClose); if emit_on_window_close { @@ -254,6 +255,7 @@ pub fn gen_sink_plan( UserId::new(session.user_id()), connection_id, dependent_relations.into_iter().collect_vec(), + secret_ref, ); if let Some(table_catalog) = &target_table_catalog { diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index bd77ea68ce8a8..75e188c086947 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -81,7 +81,7 @@ use crate::handler::HandlerArgs; use crate::optimizer::plan_node::generic::SourceNodeKind; use crate::optimizer::plan_node::{LogicalSource, ToStream, ToStreamContext}; use crate::session::SessionImpl; -use crate::utils::resolve_privatelink_in_with_option; +use crate::utils::{resolve_privatelink_in_with_option, resolve_secret_in_with_options}; use crate::{bind_data_type, build_graph, OptimizerContext, WithOptions}; pub(crate) const UPSTREAM_SOURCE_KEY: &str = "connector"; @@ -1423,6 +1423,7 @@ pub async fn bind_create_source( let mut with_properties = WithOptions::new(with_properties); let connection_id = resolve_privatelink_in_with_option(&mut with_properties, &schema_name, session)?; + let _secret_ref = resolve_secret_in_with_options(&mut with_properties, session)?; let definition: String = handler_args.normalized_sql.clone(); diff --git a/src/frontend/src/handler/drop_secret.rs b/src/frontend/src/handler/drop_secret.rs new file mode 100644 index 0000000000000..37fbd2cedd408 --- /dev/null +++ b/src/frontend/src/handler/drop_secret.rs @@ -0,0 +1,64 @@ +// Copyright 2024 RisingWave Labs +// +// 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 pgwire::pg_response::StatementType; +use risingwave_sqlparser::ast::ObjectName; + +use crate::catalog::root_catalog::SchemaPath; +use crate::error::Result; +use crate::handler::{HandlerArgs, RwPgResponse}; +use crate::Binder; + +pub async fn handle_drop_secret( + handler_args: HandlerArgs, + secret_name: ObjectName, + if_exists: bool, +) -> Result { + let session = handler_args.session; + let db_name = session.database(); + let (schema_name, secret_name) = Binder::resolve_schema_qualified_name(db_name, secret_name)?; + let search_path = session.config().search_path(); + let user_name = &session.auth_context().user_name; + let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name); + + let secret_id = { + let reader = session.env().catalog_reader().read_guard(); + let (secret, schema_name) = + match reader.get_secret_by_name(db_name, schema_path, secret_name.as_str()) { + Ok((c, s)) => (c, s), + Err(e) => { + return if if_exists { + Ok(RwPgResponse::builder(StatementType::DROP_SECRET) + .notice(format!( + "secret \"{}\" does not exist, skipping", + secret_name + )) + .into()) + } else { + Err(e.into()) + }; + } + }; + session.check_privilege_for_drop_alter(schema_name, &**secret)?; + + secret.secret_id + }; + + let catalog_writer = session.catalog_writer()?; + catalog_writer.drop_secret(secret_id).await?; + + Ok(RwPgResponse::builder(StatementType::DROP_SECRET) + .notice(format!("dropped secret \"{}\"", secret_name)) + .into()) +} diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 11c64c5bb27dd..f8beeedb19438 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -60,6 +60,7 @@ pub mod create_function; pub mod create_index; pub mod create_mv; pub mod create_schema; +pub mod create_secret; pub mod create_sink; pub mod create_source; pub mod create_sql_function; @@ -77,6 +78,7 @@ pub mod drop_function; mod drop_index; pub mod drop_mv; mod drop_schema; +pub mod drop_secret; pub mod drop_sink; pub mod drop_source; pub mod drop_subscription; @@ -257,6 +259,9 @@ pub async fn handle( Statement::CreateConnection { stmt } => { create_connection::handle_create_connection(handler_args, stmt).await } + Statement::CreateSecret { stmt } => { + create_secret::handle_create_secret(handler_args, stmt).await + } Statement::CreateFunction { or_replace, temporary, @@ -441,7 +446,8 @@ pub async fn handle( ObjectType::Schema | ObjectType::Database | ObjectType::User - | ObjectType::Connection => { + | ObjectType::Connection + | ObjectType::Secret => { bail_not_implemented!("DROP CASCADE"); } }; @@ -508,6 +514,9 @@ pub async fn handle( drop_connection::handle_drop_connection(handler_args, object_name, if_exists) .await } + ObjectType::Secret => { + drop_secret::handle_drop_secret(handler_args, object_name, if_exists).await + } } } // XXX: should we reuse Statement::Drop for DROP FUNCTION? diff --git a/src/frontend/src/handler/show.rs b/src/frontend/src/handler/show.rs index 359d556524240..f2d5186b67962 100644 --- a/src/frontend/src/handler/show.rs +++ b/src/frontend/src/handler/show.rs @@ -333,6 +333,12 @@ pub async fn handle_show_object( .iter_subscription() .map(|t| t.name.clone()) .collect(), + ShowObject::Secret { schema } => catalog_reader + .read_guard() + .get_schema_by_name(session.database(), &schema_or_default(&schema))? + .iter_secret() + .map(|t| t.name.clone()) + .collect(), ShowObject::Columns { table } => { let Ok(columns) = get_columns_from_table(&session, table.clone()) .or(get_columns_from_sink(&session, table.clone())) diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index f864f9608bdba..e5313b9601eb4 100644 --- a/src/frontend/src/observer/observer_manager.rs +++ b/src/frontend/src/observer/observer_manager.rs @@ -32,7 +32,7 @@ use risingwave_rpc_client::ComputeClientPoolRef; use tokio::sync::watch::Sender; use crate::catalog::root_catalog::Catalog; -use crate::catalog::FragmentId; +use crate::catalog::{FragmentId, SecretId}; use crate::scheduler::HummockSnapshotManagerRef; use crate::user::user_manager::UserInfoManager; use crate::user::UserInfoVersion; @@ -63,6 +63,7 @@ impl ObserverState for FrontendObserverNode { | Info::Schema(_) | Info::RelationGroup(_) | Info::Function(_) + | Info::Secret(_) | Info::Connection(_) => { self.handle_catalog_notification(resp); } @@ -142,6 +143,7 @@ impl ObserverState for FrontendObserverNode { serving_worker_slot_mappings, session_params, version, + secrets, } = snapshot; for db in databases { @@ -174,6 +176,9 @@ impl ObserverState for FrontendObserverNode { for connection in connections { catalog_guard.create_connection(&connection) } + for secret in secrets { + catalog_guard.create_secret(&secret) + } for user in users { user_guard.create_user(user) } @@ -346,6 +351,16 @@ impl FrontendObserverNode { Operation::Update => catalog_guard.update_connection(connection), _ => panic!("receive an unsupported notify {:?}", resp), }, + Info::Secret(secret) => match resp.operation() { + Operation::Add => catalog_guard.create_secret(secret), + Operation::Delete => catalog_guard.drop_secret( + secret.database_id, + secret.schema_id, + SecretId::new(secret.id), + ), + Operation::Update => catalog_guard.update_secret(secret), + _ => panic!("receive an unsupported notify {:?}", resp), + }, _ => unreachable!(), } assert!( diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 2fa728194a17d..04e6d7fb4e294 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -807,6 +807,26 @@ impl SessionImpl { } } + pub fn check_secret_name_duplicated(&self, name: ObjectName) -> Result<()> { + let db_name = self.database(); + let catalog_reader = self.env().catalog_reader().read_guard(); + let (schema_name, secret_name) = { + let (schema_name, secret_name) = Binder::resolve_schema_qualified_name(db_name, name)?; + let search_path = self.config().search_path(); + let user_name = &self.auth_context().user_name; + let schema_name = match schema_name { + Some(schema_name) => schema_name, + None => catalog_reader + .first_valid_schema(db_name, &search_path, user_name)? + .name(), + }; + (schema_name, secret_name) + }; + catalog_reader + .check_secret_name_duplicated(db_name, &schema_name, &secret_name) + .map_err(RwError::from) + } + pub fn check_connection_name_duplicated(&self, name: ObjectName) -> Result<()> { let db_name = self.database(); let catalog_reader = self.env().catalog_reader().read_guard(); diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index b3bf70f9c523f..97ab9f4eecdc9 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -64,7 +64,7 @@ use tempfile::{Builder, NamedTempFile}; use crate::catalog::catalog_service::CatalogWriter; use crate::catalog::root_catalog::Catalog; -use crate::catalog::{ConnectionId, DatabaseId, SchemaId}; +use crate::catalog::{ConnectionId, DatabaseId, SchemaId, SecretId}; use crate::error::{ErrorCode, Result}; use crate::handler::RwPgResponse; use crate::meta_client::FrontendMetaClient; @@ -373,6 +373,17 @@ impl CatalogWriter for MockCatalogWriter { unreachable!() } + async fn create_secret( + &self, + _secret_name: String, + _database_id: u32, + _schema_id: u32, + _owner_id: u32, + _payload: Vec, + ) -> Result<()> { + unreachable!() + } + async fn comment_on(&self, _comment: PbComment) -> Result<()> { unreachable!() } @@ -530,6 +541,10 @@ impl CatalogWriter for MockCatalogWriter { unreachable!() } + async fn drop_secret(&self, _secret_id: SecretId) -> Result<()> { + unreachable!() + } + async fn drop_database(&self, database_id: u32) -> Result<()> { self.catalog.write().drop_database(database_id); Ok(()) diff --git a/src/frontend/src/utils/with_options.rs b/src/frontend/src/utils/with_options.rs index b5848d124c9d3..3ee50276e5d10 100644 --- a/src/frontend/src/utils/with_options.rs +++ b/src/frontend/src/utils/with_options.rs @@ -120,6 +120,15 @@ impl WithOptions { } } +pub(crate) fn resolve_secret_in_with_options( + _with_options: &mut WithOptions, + _session: &SessionImpl, +) -> RwResult> { + // todo: implement the function and take `resolve_privatelink_in_with_option` as reference + + Ok(HashMap::new()) +} + pub(crate) fn resolve_privatelink_in_with_option( with_options: &mut WithOptions, schema_name: &Option, diff --git a/src/meta/Cargo.toml b/src/meta/Cargo.toml index 24dfecd99524e..6252d845788af 100644 --- a/src/meta/Cargo.toml +++ b/src/meta/Cargo.toml @@ -14,6 +14,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] +aes-siv = "0.7" anyhow = "1" arc-swap = "1" assert_matches = "1" @@ -21,6 +22,7 @@ async-trait = "0.1" aws-config = { workspace = true } aws-sdk-ec2 = { workspace = true } base64-url = { version = "3.0.0" } +bincode = "1.3" bytes = { version = "1", features = ["serde"] } chrono = "0.4" clap = { workspace = true } diff --git a/src/meta/model_v2/migration/src/lib.rs b/src/meta/model_v2/migration/src/lib.rs index 74968d2e3a11c..66f136b6159d1 100644 --- a/src/meta/model_v2/migration/src/lib.rs +++ b/src/meta/model_v2/migration/src/lib.rs @@ -10,6 +10,7 @@ mod m20240410_154406_session_params; mod m20240417_062305_subscription_internal_table_name; mod m20240418_142249_function_runtime; mod m20240506_112555_subscription_partial_ckpt; +mod m20240525_090457_secret; pub struct Migrator; @@ -25,6 +26,7 @@ impl MigratorTrait for Migrator { Box::new(m20240417_062305_subscription_internal_table_name::Migration), Box::new(m20240418_142249_function_runtime::Migration), Box::new(m20240506_112555_subscription_partial_ckpt::Migration), + Box::new(m20240525_090457_secret::Migration), ] } } diff --git a/src/meta/model_v2/migration/src/m20240525_090457_secret.rs b/src/meta/model_v2/migration/src/m20240525_090457_secret.rs new file mode 100644 index 0000000000000..f16bfca5ec035 --- /dev/null +++ b/src/meta/model_v2/migration/src/m20240525_090457_secret.rs @@ -0,0 +1,79 @@ +use sea_orm_migration::prelude::{Table as MigrationTable, *}; + +use crate::{assert_not_has_tables, drop_tables}; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + assert_not_has_tables!(manager, Secret); + manager + .create_table( + MigrationTable::create() + .table(Secret::Table) + .if_not_exists() + .col( + ColumnDef::new(Secret::SecretId) + .integer() + .not_null() + .primary_key(), + ) + .col(ColumnDef::new(Secret::Name).string().not_null()) + .col(ColumnDef::new(Secret::Value).binary().not_null()) + .foreign_key( + &mut ForeignKey::create() + .name("FK_secret_object_id") + .from(Secret::Table, Secret::SecretId) + .to( + crate::m20230908_072257_init::Object::Table, + crate::m20230908_072257_init::Object::Oid, + ) + .on_delete(ForeignKeyAction::Cascade) + .to_owned(), + ) + .to_owned(), + ) + .await?; + + // Add a new column to the table + manager + .alter_table( + MigrationTable::alter() + .table(Sink::Table) + .add_column(ColumnDef::new(Sink::SecretRef).json_binary()) + .to_owned(), + ) + .await?; + + Ok(()) + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + drop_tables!(manager, Secret); + manager + .alter_table( + MigrationTable::alter() + .table(Sink::Table) + .drop_column(Sink::SecretRef) + .to_owned(), + ) + .await?; + Ok(()) + } +} + +#[derive(DeriveIden)] +enum Secret { + Table, + SecretId, + Name, + Value, +} + +#[derive(DeriveIden)] +enum Sink { + Table, + SecretRef, +} diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index 87d7e3e3597f6..864c647957ddf 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -43,6 +43,7 @@ pub mod index; pub mod object; pub mod object_dependency; pub mod schema; +pub mod secret; pub mod serde_seaql_migration; pub mod session_parameter; pub mod sink; @@ -72,6 +73,7 @@ pub type IndexId = ObjectId; pub type ViewId = ObjectId; pub type FunctionId = ObjectId; pub type ConnectionId = ObjectId; +pub type SecretId = ObjectId; pub type UserId = i32; pub type PrivilegeId = i32; @@ -284,6 +286,8 @@ impl From>> for ActorUpstreamActors { } } +derive_from_json_struct!(SecretRef, HashMap); + derive_from_blob!(StreamNode, PbStreamNode); derive_from_blob!(DataType, risingwave_pb::data::PbDataType); derive_array_from_blob!( diff --git a/src/meta/model_v2/src/object.rs b/src/meta/model_v2/src/object.rs index 2b9c291f1e4fe..663f436fcbcb6 100644 --- a/src/meta/model_v2/src/object.rs +++ b/src/meta/model_v2/src/object.rs @@ -40,6 +40,8 @@ pub enum ObjectType { Connection, #[sea_orm(string_value = "SUBSCRIPTION")] Subscription, + #[sea_orm(string_value = "SECRET")] + Secret, } impl ObjectType { @@ -55,6 +57,7 @@ impl ObjectType { ObjectType::Function => "function", ObjectType::Connection => "connection", ObjectType::Subscription => "subscription", + ObjectType::Secret => "secret", } } } diff --git a/src/meta/model_v2/src/prelude.rs b/src/meta/model_v2/src/prelude.rs index 7d3c8cde7acb1..b17eae112aef0 100644 --- a/src/meta/model_v2/src/prelude.rs +++ b/src/meta/model_v2/src/prelude.rs @@ -32,6 +32,7 @@ pub use super::index::Entity as Index; pub use super::object::Entity as Object; pub use super::object_dependency::Entity as ObjectDependency; pub use super::schema::Entity as Schema; +pub use super::secret::Entity as Secret; pub use super::session_parameter::Entity as SessionParameter; pub use super::sink::Entity as Sink; pub use super::source::Entity as Source; diff --git a/src/meta/model_v2/src/secret.rs b/src/meta/model_v2/src/secret.rs new file mode 100644 index 0000000000000..af3590dd0de58 --- /dev/null +++ b/src/meta/model_v2/src/secret.rs @@ -0,0 +1,57 @@ +// Copyright 2024 RisingWave Labs +// +// 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 risingwave_pb::catalog::PbSecret; +use sea_orm::entity::prelude::*; +use sea_orm::ActiveValue::Set; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +#[sea_orm(table_name = "secret")] +pub struct Model { + #[sea_orm(primary_key, auto_increment = false)] + pub secret_id: i32, + pub name: String, + #[sea_orm(column_type = "Binary(BlobSize::Blob(None))")] + pub value: Vec, +} + +#[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] +pub enum Relation { + #[sea_orm( + belongs_to = "super::object::Entity", + from = "Column::SecretId", + to = "super::object::Column::Oid", + on_update = "NoAction", + on_delete = "Cascade" + )] + Object, +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::Object.def() + } +} + +impl ActiveModelBehavior for ActiveModel {} + +impl From for ActiveModel { + fn from(secret: PbSecret) -> Self { + Self { + secret_id: Set(secret.id as _), + name: Set(secret.name), + value: Set(secret.value), + } + } +} diff --git a/src/meta/model_v2/src/sink.rs b/src/meta/model_v2/src/sink.rs index eafa1beee92f2..78d0806f98a5e 100644 --- a/src/meta/model_v2/src/sink.rs +++ b/src/meta/model_v2/src/sink.rs @@ -18,8 +18,8 @@ use sea_orm::ActiveValue::Set; use serde::{Deserialize, Serialize}; use crate::{ - ColumnCatalogArray, ColumnOrderArray, ConnectionId, I32Array, Property, SinkFormatDesc, SinkId, - TableId, + ColumnCatalogArray, ColumnOrderArray, ConnectionId, I32Array, Property, SecretRef, + SinkFormatDesc, SinkId, TableId, }; #[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum, Serialize, Deserialize)] @@ -72,6 +72,8 @@ pub struct Model { pub sink_from_name: String, pub sink_format_desc: Option, pub target_table: Option, + // `secret_ref` stores a json string, mapping from property name to secret id. + pub secret_ref: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -127,6 +129,7 @@ impl From for ActiveModel { sink_from_name: Set(pb_sink.sink_from_name), sink_format_desc: Set(pb_sink.format_desc.as_ref().map(|x| x.into())), target_table: Set(pb_sink.target_table.map(|x| x as _)), + secret_ref: Set(Some(SecretRef::from(pb_sink.secret_ref))), } } } diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index 21cc0c67860b0..0e49f0805bf1b 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -379,6 +379,7 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { .developer .max_trivial_move_task_count_per_loop, max_get_task_probe_times: config.meta.developer.max_get_task_probe_times, + secret_store_private_key: config.meta.secret_store_private_key, }, config.system.into_init_system_params(), Default::default(), diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 7b7d4f5a7d092..2e4ba23e02d8f 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -25,7 +25,7 @@ use risingwave_pb::catalog::connection::private_link_service::{ }; use risingwave_pb::catalog::connection::PbPrivateLinkService; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; -use risingwave_pb::catalog::{connection, Comment, Connection, CreateType}; +use risingwave_pb::catalog::{connection, Comment, Connection, CreateType, Secret}; use risingwave_pb::ddl_service::ddl_service_server::DdlService; use risingwave_pb::ddl_service::drop_table_request::PbSourceId; use risingwave_pb::ddl_service::*; @@ -148,6 +148,41 @@ impl DdlService for DdlServiceImpl { })) } + async fn create_secret( + &self, + request: Request, + ) -> Result, Status> { + let req = request.into_inner(); + let pb_secret = Secret { + id: 0, + name: req.get_name().clone(), + database_id: req.get_database_id(), + value: req.get_value().clone(), + owner: req.get_owner_id(), + schema_id: req.get_schema_id(), + }; + let version = self + .ddl_controller + .run_command(DdlCommand::CreateSecret(pb_secret)) + .await?; + + Ok(Response::new(CreateSecretResponse { version })) + } + + async fn drop_secret( + &self, + request: Request, + ) -> Result, Status> { + let req = request.into_inner(); + let secret_id = req.get_secret_id(); + let version = self + .ddl_controller + .run_command(DdlCommand::DropSecret(secret_id)) + .await?; + + Ok(Response::new(DropSecretResponse { version })) + } + async fn create_schema( &self, request: Request, diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index f3fec987c2bf1..e4a8d298e0788 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -80,6 +80,7 @@ impl NotificationServiceImpl { views, functions, connections, + secrets, ) = catalog_guard.database.get_catalog(); let users = catalog_guard.user.list_users(); let notification_version = self.env.notification_manager().current_version().await; @@ -95,6 +96,7 @@ impl NotificationServiceImpl { views, functions, connections, + secrets, ), users, notification_version, @@ -114,6 +116,7 @@ impl NotificationServiceImpl { views, functions, connections, + secrets, ), users, ) = catalog_guard.snapshot().await?; @@ -130,6 +133,7 @@ impl NotificationServiceImpl { views, functions, connections, + secrets, ), users, notification_version, @@ -237,6 +241,7 @@ impl NotificationServiceImpl { views, functions, connections, + secrets, ), users, catalog_version, @@ -271,6 +276,7 @@ impl NotificationServiceImpl { subscriptions, functions, connections, + secrets, users, nodes, hummock_snapshot, diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index cb3e386c34fc5..200736725cd07 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -28,16 +28,16 @@ use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::table::TableType; use risingwave_meta_model_v2::{ actor, connection, database, fragment, function, index, object, object_dependency, schema, - sink, source, streaming_job, subscription, table, user_privilege, view, ActorId, + secret, sink, source, streaming_job, subscription, table, user_privilege, view, ActorId, ActorUpstreamActors, ColumnCatalogArray, ConnectionId, CreateType, DatabaseId, FragmentId, FunctionId, I32Array, IndexId, JobStatus, ObjectId, PrivateLinkService, Property, SchemaId, - SinkId, SourceId, StreamNode, StreamSourceInfo, StreamingParallelism, SubscriptionId, TableId, - UserId, ViewId, + SecretId, SinkId, SourceId, StreamNode, StreamSourceInfo, StreamingParallelism, SubscriptionId, + TableId, UserId, ViewId, }; use risingwave_pb::catalog::subscription::SubscriptionState; use risingwave_pb::catalog::table::PbTableType; use risingwave_pb::catalog::{ - PbComment, PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, + PbComment, PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSecret, PbSink, PbSource, PbSubscription, PbTable, PbView, }; use risingwave_pb::meta::cancel_creating_jobs_request::PbCreatingJobInfo; @@ -66,10 +66,10 @@ use crate::controller::rename::{alter_relation_rename, alter_relation_rename_ref use crate::controller::utils::{ check_connection_name_duplicate, check_database_name_duplicate, check_function_signature_duplicate, check_relation_name_duplicate, check_schema_name_duplicate, - ensure_object_id, ensure_object_not_refer, ensure_schema_empty, ensure_user_id, - get_fragment_mappings_by_jobs, get_parallel_unit_to_worker_map, get_referring_objects, - get_referring_objects_cascade, get_user_privilege, list_user_info_by_ids, - resolve_source_register_info_for_jobs, PartialObject, + check_secret_name_duplicate, ensure_object_id, ensure_object_not_refer, ensure_schema_empty, + ensure_user_id, get_fragment_mappings_by_jobs, get_parallel_unit_to_worker_map, + get_referring_objects, get_referring_objects_cascade, get_user_privilege, + list_user_info_by_ids, resolve_source_register_info_for_jobs, PartialObject, }; use crate::controller::ObjectModel; use crate::manager::{Catalog, MetaSrvEnv, NotificationVersion, IGNORED_NOTIFICATION_VERSION}; @@ -1100,6 +1100,88 @@ impl CatalogController { Ok(version) } + pub async fn create_secret(&self, mut pb_secret: PbSecret) -> MetaResult { + let inner = self.inner.write().await; + let owner_id = pb_secret.owner as _; + let txn = inner.db.begin().await?; + ensure_user_id(owner_id, &txn).await?; + ensure_object_id(ObjectType::Database, pb_secret.database_id as _, &txn).await?; + ensure_object_id(ObjectType::Schema, pb_secret.schema_id as _, &txn).await?; + check_secret_name_duplicate(&pb_secret, &txn).await?; + + let secret_obj = Self::create_object( + &txn, + ObjectType::Secret, + owner_id, + Some(pb_secret.database_id as _), + Some(pb_secret.schema_id as _), + ) + .await?; + pb_secret.id = secret_obj.oid as _; + let secret: secret::ActiveModel = pb_secret.clone().into(); + Secret::insert(secret).exec(&txn).await?; + + txn.commit().await?; + + let version = self + .notify_frontend( + NotificationOperation::Add, + NotificationInfo::Secret(pb_secret), + ) + .await; + Ok(version) + } + + pub async fn get_secret_by_id(&self, secret_id: SecretId) -> MetaResult { + let inner = self.inner.read().await; + let (secret, obj) = Secret::find_by_id(secret_id) + .find_also_related(Object) + .one(&inner.db) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("secret", secret_id))?; + Ok(ObjectModel(secret, obj.unwrap()).into()) + } + + pub async fn drop_secret(&self, secret_id: SecretId) -> MetaResult { + let inner = self.inner.write().await; + let txn = inner.db.begin().await?; + let (secret, secret_obj) = Secret::find_by_id(secret_id) + .find_also_related(Object) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("secret", secret_id))?; + ensure_object_not_refer(ObjectType::Secret, secret_id, &txn).await?; + + // Find affect users with privileges on the connection. + let to_update_user_ids: Vec = UserPrivilege::find() + .select_only() + .distinct() + .column(user_privilege::Column::UserId) + .filter(user_privilege::Column::Oid.eq(secret_id)) + .into_tuple() + .all(&txn) + .await?; + + let res = Object::delete_by_id(secret_id).exec(&txn).await?; + if res.rows_affected == 0 { + return Err(MetaError::catalog_id_not_found("secret", secret_id)); + } + let user_infos = list_user_info_by_ids(to_update_user_ids, &txn).await?; + + txn.commit().await?; + + let pb_secret: PbSecret = ObjectModel(secret, secret_obj.unwrap()).into(); + + self.notify_users_update(user_infos).await; + let version = self + .notify_frontend( + NotificationOperation::Delete, + NotificationInfo::Secret(pb_secret), + ) + .await; + Ok(version) + } + pub async fn create_connection( &self, mut pb_connection: PbConnection, @@ -2804,6 +2886,7 @@ impl CatalogControllerInner { let views = self.list_views().await?; let functions = self.list_functions().await?; let connections = self.list_connections().await?; + let secrets = self.list_secrets().await?; let users = self.list_users().await?; @@ -2819,6 +2902,7 @@ impl CatalogControllerInner { views, functions, connections, + secrets, ), users, )) @@ -3037,6 +3121,17 @@ impl CatalogControllerInner { .collect()) } + async fn list_secrets(&self) -> MetaResult> { + let secret_objs = Secret::find() + .find_also_related(Object) + .all(&self.db) + .await?; + Ok(secret_objs + .into_iter() + .map(|(secret, obj)| ObjectModel(secret, obj.unwrap()).into()) + .collect()) + } + async fn list_functions(&self) -> MetaResult> { let func_objs = Function::find() .find_also_related(Object) diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 7e9f20f7557d6..43d7efed1ba58 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -12,10 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; + use anyhow::anyhow; use risingwave_common::util::epoch::Epoch; use risingwave_meta_model_v2::{ - connection, database, function, index, object, schema, sink, source, subscription, table, view, + connection, database, function, index, object, schema, secret, sink, source, subscription, + table, view, }; use risingwave_pb::catalog::connection::PbInfo as PbConnectionInfo; use risingwave_pb::catalog::source::PbOptionalAssociatedTableId; @@ -23,7 +26,8 @@ use risingwave_pb::catalog::subscription::PbSubscriptionState; use risingwave_pb::catalog::table::{PbOptionalAssociatedSourceId, PbTableType}; use risingwave_pb::catalog::{ PbConnection, PbCreateType, PbDatabase, PbFunction, PbHandleConflictBehavior, PbIndex, - PbSchema, PbSink, PbSinkType, PbSource, PbStreamJobStatus, PbSubscription, PbTable, PbView, + PbSchema, PbSecret, PbSink, PbSinkType, PbSource, PbStreamJobStatus, PbSubscription, PbTable, + PbView, }; use sea_orm::{DatabaseConnection, ModelTrait}; @@ -82,6 +86,19 @@ impl From> for PbDatabase { } } +impl From> for PbSecret { + fn from(value: ObjectModel) -> Self { + Self { + id: value.0.secret_id as _, + name: value.0.name, + database_id: value.1.database_id.unwrap() as _, + value: value.0.value, + owner: value.1.owner_id as _, + schema_id: value.1.schema_id.unwrap() as _, + } + } +} + impl From> for PbSchema { fn from(value: ObjectModel) -> Self { Self { @@ -184,6 +201,10 @@ impl From> for PbSource { impl From> for PbSink { fn from(value: ObjectModel) -> Self { + let mut secret_ref_hashmap: HashMap = HashMap::new(); + if let Some(secret_ref) = value.0.secret_ref { + secret_ref_hashmap = secret_ref.into_inner(); + } Self { id: value.0.sink_id as _, schema_id: value.1.schema_id.unwrap() as _, @@ -213,6 +234,7 @@ impl From> for PbSink { initialized_at_cluster_version: value.1.initialized_at_cluster_version, created_at_cluster_version: value.1.created_at_cluster_version, create_type: PbCreateType::Foreground as _, + secret_ref: secret_ref_hashmap, } } } diff --git a/src/meta/src/controller/utils.rs b/src/meta/src/controller/utils.rs index b654050708546..b98788248a115 100644 --- a/src/meta/src/controller/utils.rs +++ b/src/meta/src/controller/utils.rs @@ -24,11 +24,11 @@ use risingwave_meta_model_v2::object::ObjectType; use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::{ actor, actor_dispatcher, connection, database, fragment, function, index, object, - object_dependency, schema, sink, source, subscription, table, user, user_privilege, view, - worker_property, ActorId, DataTypeArray, DatabaseId, FragmentId, FragmentVnodeMapping, + object_dependency, schema, secret, sink, source, subscription, table, user, user_privilege, + view, worker_property, ActorId, DataTypeArray, DatabaseId, FragmentId, FragmentVnodeMapping, I32Array, ObjectId, PrivilegeId, SchemaId, SourceId, StreamNode, UserId, WorkerId, }; -use risingwave_pb::catalog::{PbConnection, PbFunction, PbSubscription}; +use risingwave_pb::catalog::{PbConnection, PbFunction, PbSecret, PbSubscription}; use risingwave_pb::meta::{PbFragmentParallelUnitMapping, PbFragmentWorkerSlotMapping}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{PbFragmentTypeFlag, PbStreamNode, StreamSource}; @@ -411,6 +411,27 @@ where Ok(()) } +pub async fn check_secret_name_duplicate(pb_secret: &PbSecret, db: &C) -> MetaResult<()> +where + C: ConnectionTrait, +{ + let count = Secret::find() + .inner_join(Object) + .filter( + object::Column::DatabaseId + .eq(pb_secret.database_id as DatabaseId) + .and(object::Column::SchemaId.eq(pb_secret.schema_id as SchemaId)) + .and(secret::Column::Name.eq(&pb_secret.name)), + ) + .count(db) + .await?; + if count > 0 { + assert_eq!(count, 1); + return Err(MetaError::catalog_duplicated("secret", &pb_secret.name)); + } + Ok(()) +} + pub async fn check_subscription_name_duplicate( pb_subscription: &PbSubscription, db: &C, @@ -762,6 +783,7 @@ where ObjectType::Function => PbObject::FunctionId(oid), ObjectType::Connection => unreachable!("connection is not supported yet"), ObjectType::Subscription => PbObject::SubscriptionId(oid), + ObjectType::Secret => unreachable!("secret is not supported yet"), }; PbGrantPrivilege { action_with_opts: vec![PbActionWithGrantOption { diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index 5b1b24e82de21..1cea1e0c393d8 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -21,15 +21,15 @@ use risingwave_common::catalog::TableOption; use risingwave_pb::catalog::subscription::PbSubscriptionState; use risingwave_pb::catalog::table::TableType; use risingwave_pb::catalog::{ - Connection, CreateType, Database, Function, Index, PbStreamJobStatus, Schema, Sink, Source, - StreamJobStatus, Subscription, Table, View, + Connection, CreateType, Database, Function, Index, PbStreamJobStatus, Schema, Secret, Sink, + Source, StreamJobStatus, Subscription, Table, View, }; use risingwave_pb::data::DataType; use risingwave_pb::user::grant_privilege::PbObject; use super::{ - ConnectionId, DatabaseId, FunctionId, RelationId, SchemaId, SinkId, SourceId, SubscriptionId, - ViewId, + ConnectionId, DatabaseId, FunctionId, RelationId, SchemaId, SecretId, SinkId, SourceId, + SubscriptionId, ViewId, }; use crate::manager::{IndexId, MetaSrvEnv, TableId, UserId}; use crate::model::MetadataModel; @@ -46,6 +46,7 @@ pub type Catalog = ( Vec, Vec, Vec, + Vec, ); type DatabaseKey = String; @@ -76,10 +77,15 @@ pub struct DatabaseManager { pub(super) functions: BTreeMap, /// Cached connection information. pub(super) connections: BTreeMap, + /// Cached secret information. + pub(super) secrets: BTreeMap, /// Relation reference count mapping. // TODO(zehua): avoid key conflicts after distinguishing table's and source's id generator. pub(super) relation_ref_count: HashMap, + + /// Secret reference count mapping + pub(super) secret_ref_count: HashMap, // In-progress creation tracker. pub(super) in_progress_creation_tracker: HashSet, // In-progress creating streaming job tracker: this is a temporary workaround to avoid clean up @@ -101,8 +107,10 @@ impl DatabaseManager { let functions = Function::list(env.meta_store().as_kv()).await?; let connections = Connection::list(env.meta_store().as_kv()).await?; let subscriptions = Subscription::list(env.meta_store().as_kv()).await?; + let secrets = Secret::list(env.meta_store().as_kv()).await?; let mut relation_ref_count = HashMap::new(); + let mut _secret_ref_count = HashMap::new(); let databases = BTreeMap::from_iter( databases @@ -129,6 +137,7 @@ impl DatabaseManager { .or_default() += 1; (subscription.id, subscription) })); + let secrets = BTreeMap::from_iter(secrets.into_iter().map(|secret| (secret.id, secret))); let indexes = BTreeMap::from_iter(indexes.into_iter().map(|index| (index.id, index))); let tables = BTreeMap::from_iter(tables.into_iter().map(|table| { for depend_relation_id in &table.dependent_relations { @@ -145,6 +154,8 @@ impl DatabaseManager { let functions = BTreeMap::from_iter(functions.into_iter().map(|f| (f.id, f))); let connections = BTreeMap::from_iter(connections.into_iter().map(|c| (c.id, c))); + // todo: scan over stream source info and sink to update secret ref count `_secret_ref_count` + Ok(Self { databases, schemas, @@ -157,6 +168,8 @@ impl DatabaseManager { functions, connections, relation_ref_count, + secrets, + secret_ref_count: _secret_ref_count, in_progress_creation_tracker: HashSet::default(), in_progress_creation_streaming_job: HashMap::default(), in_progress_creating_tables: HashMap::default(), @@ -200,6 +213,7 @@ impl DatabaseManager { self.views.values().cloned().collect_vec(), self.functions.values().cloned().collect_vec(), self.connections.values().cloned().collect_vec(), + self.secrets.values().cloned().collect_vec(), ) } @@ -292,6 +306,16 @@ impl DatabaseManager { } } + pub fn check_secret_name_duplicated(&self, secret_key: &RelationKey) -> MetaResult<()> { + if self.secrets.values().any(|x| { + x.database_id == secret_key.0 && x.schema_id == secret_key.1 && x.name.eq(&secret_key.2) + }) { + Err(MetaError::catalog_duplicated("secret", &secret_key.2)) + } else { + Ok(()) + } + } + pub fn list_databases(&self) -> Vec { self.databases.values().cloned().collect_vec() } @@ -467,6 +491,22 @@ impl DatabaseManager { } } + pub fn increase_secret_ref_count(&mut self, secret_id: SecretId) { + *self.secret_ref_count.entry(secret_id).or_insert(0) += 1; + } + + pub fn decrease_secret_ref_count(&mut self, secret_id: SecretId) { + match self.secret_ref_count.entry(secret_id) { + Entry::Occupied(mut o) => { + *o.get_mut() -= 1; + if *o.get() == 0 { + o.remove_entry(); + } + } + Entry::Vacant(_) => unreachable!(), + } + } + pub fn has_creation_in_database(&self, database_id: DatabaseId) -> bool { self.in_progress_creation_tracker .iter() diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 9259a32eb55cf..824d656e506c0 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -36,7 +36,7 @@ use risingwave_pb::catalog::subscription::PbSubscriptionState; use risingwave_pb::catalog::table::{OptionalAssociatedSourceId, TableType}; use risingwave_pb::catalog::{ Comment, Connection, CreateType, Database, Function, Index, PbSource, PbStreamJobStatus, - Schema, Sink, Source, StreamJobStatus, Subscription, Table, View, + Schema, Secret, Sink, Source, StreamJobStatus, Subscription, Table, View, }; use risingwave_pb::ddl_service::{alter_owner_request, alter_set_schema_request}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -63,6 +63,7 @@ pub type RelationId = u32; pub type IndexId = u32; pub type ViewId = u32; pub type FunctionId = u32; +pub type SecretId = u32; pub type UserId = u32; pub type ConnectionId = u32; @@ -328,6 +329,7 @@ impl CatalogManager { let mut users = BTreeMapTransaction::new(&mut user_core.user_info); let mut functions = BTreeMapTransaction::new(&mut database_core.functions); let mut connections = BTreeMapTransaction::new(&mut database_core.connections); + let mut secrets = BTreeMapTransaction::new(&mut database_core.secrets); /// `drop_by_database_id` provides a wrapper for dropping relations by database id, it will /// return the relation ids that dropped. @@ -360,6 +362,7 @@ impl CatalogManager { let views_to_drop = drop_by_database_id!(views, database_id); let functions_to_drop = drop_by_database_id!(functions, database_id); let connections_to_drop = drop_by_database_id!(connections, database_id); + let secrets_to_drop = drop_by_database_id!(secrets, database_id); connections_dropped = connections_to_drop.clone(); let objects = std::iter::once(Object::DatabaseId(database_id)) @@ -421,6 +424,7 @@ impl CatalogManager { .iter() .map(|connection| connection.owner), ) + .chain(secrets_to_drop.iter().map(|secret| secret.owner)) .for_each(|owner_id| user_core.decrease_ref(owner_id)); // Update relation ref count. @@ -478,6 +482,57 @@ impl CatalogManager { } } + pub async fn create_secret(&self, secret: Secret) -> MetaResult { + let core = &mut *self.core.lock().await; + let database_core = &mut core.database; + let user_core = &mut core.user; + database_core.ensure_database_id(secret.database_id)?; + database_core.ensure_schema_id(secret.schema_id)?; + #[cfg(not(test))] + user_core.ensure_user_id(secret.owner)?; + let key = ( + secret.database_id as DatabaseId, + secret.schema_id as SchemaId, + secret.name.clone(), + ); + database_core.check_secret_name_duplicated(&key)?; + + let secret_id = secret.id; + let mut secret_entry = BTreeMapTransaction::new(&mut database_core.secrets); + secret_entry.insert(secret_id, secret.to_owned()); + commit_meta!(self, secret_entry)?; + + user_core.increase_ref(secret.owner); + + let version = self + .notify_frontend(Operation::Add, Info::Secret(secret)) + .await; + Ok(version) + } + + pub async fn drop_secret(&self, secret_id: SecretId) -> MetaResult { + let core = &mut *self.core.lock().await; + let database_core = &mut core.database; + let user_core = &mut core.user; + let mut secrets = BTreeMapTransaction::new(&mut database_core.secrets); + + // todo: impl a ref count check for secret + // if secret is used by other relations, not found in the catalog or do not have the privilege to drop, return error + // else: commit the change and notify frontend + + let secret = secrets + .remove(secret_id) + .ok_or_else(|| anyhow!("secret not found"))?; + + commit_meta!(self, secrets)?; + user_core.decrease_ref(secret.owner); + + let version = self + .notify_frontend(Operation::Delete, Info::Secret(secret)) + .await; + Ok(version) + } + pub async fn create_connection( &self, connection: Connection, diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index a8e2909a11bf7..af7218219afc4 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -280,6 +280,9 @@ pub struct MetaOpts { pub compact_task_table_size_partition_threshold_low: u64, pub compact_task_table_size_partition_threshold_high: u64, + + // The private key for the secret store, used when the secret is stored in the meta. + pub secret_store_private_key: Vec, } impl MetaOpts { @@ -340,6 +343,7 @@ impl MetaOpts { object_store_config: ObjectStoreConfig::default(), max_trivial_move_task_count_per_loop: 256, max_get_task_probe_times: 5, + secret_store_private_key: "demo-secret-private-key".as_bytes().to_vec(), } } } diff --git a/src/meta/src/manager/id.rs b/src/meta/src/manager/id.rs index 7fbde6d655e83..023483116fdc8 100644 --- a/src/meta/src/manager/id.rs +++ b/src/meta/src/manager/id.rs @@ -137,6 +137,8 @@ pub mod IdCategory { pub const CompactionGroup: IdCategoryType = 15; pub const Function: IdCategoryType = 16; pub const Connection: IdCategoryType = 17; + + pub const Secret: IdCategoryType = 18; } pub type IdGeneratorManagerRef = Arc; @@ -160,6 +162,7 @@ pub struct IdGeneratorManager { parallel_unit: Arc, compaction_group: Arc, connection: Arc, + secret: Arc, } impl IdGeneratorManager { @@ -209,6 +212,7 @@ impl IdGeneratorManager { connection: Arc::new( StoredIdGenerator::new(meta_store.clone(), "connection", None).await, ), + secret: Arc::new(StoredIdGenerator::new(meta_store.clone(), "secret", None).await), } } @@ -230,6 +234,7 @@ impl IdGeneratorManager { IdCategory::HummockCompactionTask => &self.hummock_compaction_task, IdCategory::CompactionGroup => &self.compaction_group, IdCategory::Connection => &self.connection, + IdCategory::Secret => &self.secret, _ => unreachable!(), } } diff --git a/src/meta/src/model/catalog.rs b/src/meta/src/model/catalog.rs index 8f762255d60b7..c11be01d1a599 100644 --- a/src/meta/src/model/catalog.rs +++ b/src/meta/src/model/catalog.rs @@ -13,7 +13,7 @@ // limitations under the License. use risingwave_pb::catalog::{ - Connection, Database, Function, Index, Schema, Sink, Source, Subscription, Table, View, + Connection, Database, Function, Index, Schema, Secret, Sink, Source, Subscription, Table, View, }; use crate::model::{MetadataModel, MetadataModelResult}; @@ -38,6 +38,8 @@ const CATALOG_SCHEMA_CF_NAME: &str = "cf/catalog_schema"; const CATALOG_DATABASE_CF_NAME: &str = "cf/catalog_database"; /// Column family name for database catalog. const CATALOG_SUBSCRIPTION_CF_NAME: &str = "cf/catalog_subscription"; +/// Column family name for secret catalog. +const CATALOG_SECRET_CF_NAME: &str = "cf/catalog_secret"; macro_rules! impl_model_for_catalog { ($name:ident, $cf:ident, $key_ty:ty, $key_fn:ident) => { @@ -74,6 +76,7 @@ impl_model_for_catalog!(Table, CATALOG_TABLE_CF_NAME, u32, get_id); impl_model_for_catalog!(Schema, CATALOG_SCHEMA_CF_NAME, u32, get_id); impl_model_for_catalog!(Database, CATALOG_DATABASE_CF_NAME, u32, get_id); impl_model_for_catalog!(Subscription, CATALOG_SUBSCRIPTION_CF_NAME, u32, get_id); +impl_model_for_catalog!(Secret, CATALOG_SECRET_CF_NAME, u32, get_id); #[cfg(test)] mod tests { diff --git a/src/meta/src/model/mod.rs b/src/meta/src/model/mod.rs index 0f20fb482a654..e87251ee6d413 100644 --- a/src/meta/src/model/mod.rs +++ b/src/meta/src/model/mod.rs @@ -177,6 +177,7 @@ macro_rules! for_all_metadata_models { { risingwave_pb::user::UserInfo }, { risingwave_pb::catalog::Function }, { risingwave_pb::catalog::Connection }, + { risingwave_pb::catalog::Secret }, // These items need not be included in a meta snapshot. { crate::model::cluster::Worker }, { risingwave_pb::hummock::CompactTaskAssignment }, diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index cd448dc51c880..f83e1420eb9c5 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -18,9 +18,12 @@ use std::num::NonZeroUsize; use std::sync::Arc; use std::time::Duration; +use aes_siv::aead::generic_array::GenericArray; +use aes_siv::aead::Aead; +use aes_siv::{Aes128SivAead, KeyInit}; use anyhow::Context; use itertools::Itertools; -use rand::Rng; +use rand::{Rng, RngCore}; use risingwave_common::config::DefaultParallelism; use risingwave_common::hash::{ParallelUnitMapping, VirtualNode}; use risingwave_common::system_param::reader::SystemParamsRead; @@ -45,7 +48,7 @@ use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ connection, Comment, Connection, CreateType, Database, Function, PbSource, PbTable, Schema, - Sink, Source, Subscription, Table, View, + Secret, Sink, Source, Subscription, Table, View, }; use risingwave_pb::ddl_service::alter_owner_request::Object; use risingwave_pb::ddl_service::{ @@ -58,6 +61,7 @@ use risingwave_pb::stream_plan::{ Dispatcher, DispatcherType, FragmentTypeFlag, MergeNode, PbStreamFragmentGraph, StreamFragmentGraph as StreamFragmentGraphProto, }; +use serde::{Deserialize, Serialize}; use thiserror_ext::AsReport; use tokio::sync::Semaphore; use tokio::time::sleep; @@ -65,12 +69,13 @@ use tracing::log::warn; use tracing::Instrument; use crate::barrier::BarrierManagerRef; +use crate::error::MetaErrorInner; use crate::manager::{ CatalogManagerRef, ConnectionId, DatabaseId, FragmentManagerRef, FunctionId, IdCategory, IdCategoryType, IndexId, LocalNotification, MetaSrvEnv, MetadataManager, MetadataManagerV1, - NotificationVersion, RelationIdEnum, SchemaId, SinkId, SourceId, StreamingClusterInfo, - StreamingJob, StreamingJobDiscriminants, SubscriptionId, TableId, UserId, ViewId, - IGNORED_NOTIFICATION_VERSION, + NotificationVersion, RelationIdEnum, SchemaId, SecretId, SinkId, SourceId, + StreamingClusterInfo, StreamingJob, StreamingJobDiscriminants, SubscriptionId, TableId, UserId, + ViewId, IGNORED_NOTIFICATION_VERSION, }; use crate::model::{FragmentId, StreamContext, TableFragments, TableParallelism}; use crate::rpc::cloud_provider::AwsEc2Client; @@ -148,11 +153,19 @@ pub enum DdlCommand { AlterSetSchema(alter_set_schema_request::Object, SchemaId), CreateConnection(Connection), DropConnection(ConnectionId), + CreateSecret(Secret), + DropSecret(SecretId), CommentOn(Comment), CreateSubscription(Subscription), DropSubscription(SubscriptionId, DropMode), } +#[derive(Deserialize, Serialize)] +struct SecretEncryption { + nonce: [u8; 16], + ciphertext: Vec, +} + impl DdlCommand { fn allow_in_recovery(&self) -> bool { match self { @@ -162,7 +175,9 @@ impl DdlCommand { | DdlCommand::DropFunction(_) | DdlCommand::DropView(_, _) | DdlCommand::DropStreamingJob(_, _, _) - | DdlCommand::DropConnection(_) => true, + | DdlCommand::DropConnection(_) + | DdlCommand::DropSecret(_) => true, + // Simply ban all other commands in recovery. _ => false, } @@ -331,6 +346,8 @@ impl DdlController { DdlCommand::DropConnection(connection_id) => { ctrl.drop_connection(connection_id).await } + DdlCommand::CreateSecret(secret) => ctrl.create_secret(secret).await, + DdlCommand::DropSecret(secret_id) => ctrl.drop_secret(secret_id).await, DdlCommand::AlterSourceColumn(source) => ctrl.alter_source_column(source).await, DdlCommand::CommentOn(comment) => ctrl.comment_on(comment).await, DdlCommand::CreateSubscription(subscription) => { @@ -609,6 +626,57 @@ impl DdlController { } } + async fn create_secret(&self, mut secret: Secret) -> MetaResult { + // The 'secret' part of the request we receive from the frontend is in plaintext; + // here, we need to encrypt it before storing it in the catalog. + + let encrypted_payload = { + let data = secret.get_value().as_slice(); + let key = self.env.opts.secret_store_private_key.as_slice(); + let encrypt_key = { + let mut k = key[..(std::cmp::min(key.len(), 32))].to_vec(); + k.resize_with(32, || 0); + k + }; + + let mut rng = rand::thread_rng(); + let mut nonce: [u8; 16] = [0; 16]; + rng.fill_bytes(&mut nonce); + let nonce_array = GenericArray::from_slice(&nonce); + let cipher = Aes128SivAead::new(encrypt_key.as_slice().into()); + + let ciphertext = cipher.encrypt(nonce_array, data).map_err(|e| { + MetaError::from(MetaErrorInner::InvalidParameter(format!( + "failed to encrypt secret {}: {:?}", + secret.name, e + ))) + })?; + bincode::serialize(&SecretEncryption { nonce, ciphertext }).map_err(|e| { + MetaError::from(MetaErrorInner::InvalidParameter(format!( + "failed to serialize secret {}: {:?}", + secret.name, + e.as_report() + ))) + })? + }; + secret.value = encrypted_payload; + + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + secret.id = self.gen_unique_id::<{ IdCategory::Secret }>().await?; + mgr.catalog_manager.create_secret(secret).await + } + MetadataManager::V2(mgr) => mgr.catalog_controller.create_secret(secret).await, + } + } + + async fn drop_secret(&self, secret_id: SecretId) -> MetaResult { + match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.drop_secret(secret_id).await, + MetadataManager::V2(mgr) => mgr.catalog_controller.drop_secret(secret_id as _).await, + } + } + pub(crate) async fn delete_vpc_endpoint(&self, connection: &Connection) -> MetaResult<()> { // delete AWS vpc endpoint if let Some(connection::Info::PrivateLinkService(svc)) = &connection.info @@ -908,7 +976,7 @@ impl DdlController { ctx, internal_tables, ) - .await + .await } (CreateType::Background, &StreamingJob::MaterializedView(_)) => { let ctrl = self.clone(); @@ -1931,20 +1999,20 @@ impl DdlController { // Map the column indices in the dispatchers with the given mapping. let downstream_fragments = self.metadata_manager.get_downstream_chain_fragments(id).await? - .into_iter() - .map(|(d, f)| - if let Some(mapping) = &table_col_index_mapping { - Some((mapping.rewrite_dispatch_strategy(&d)?, f)) - } else { - Some((d, f)) - }) - .collect::>() - .ok_or_else(|| { - // The `rewrite` only fails if some column is dropped. - MetaError::invalid_parameter( - "unable to drop the column due to being referenced by downstream materialized views or sinks", - ) - })?; + .into_iter() + .map(|(d, f)| + if let Some(mapping) = &table_col_index_mapping { + Some((mapping.rewrite_dispatch_strategy(&d)?, f)) + } else { + Some((d, f)) + }) + .collect::>() + .ok_or_else(|| { + // The `rewrite` only fails if some column is dropped. + MetaError::invalid_parameter( + "unable to drop the column due to being referenced by downstream materialized views or sinks", + ) + })?; let complete_graph = CompleteStreamFragmentGraph::with_downstreams( fragment_graph, diff --git a/src/prost/build.rs b/src/prost/build.rs index e031e5cfb01ae..67284d844cc3e 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -53,6 +53,7 @@ fn main() -> Result<(), Box> { "task_service", "telemetry", "user", + "secret", ]; let protos: Vec = proto_files .iter() diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index d4f0359fadab2..27d0523b84115 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -56,7 +56,7 @@ pub mod batch_plan; #[cfg_attr(madsim, path = "sim/task_service.rs")] pub mod task_service; #[rustfmt::skip] -#[cfg_attr(madsim, path="sim/connector_service.rs")] +#[cfg_attr(madsim, path = "sim/connector_service.rs")] pub mod connector_service; #[rustfmt::skip] #[cfg_attr(madsim, path = "sim/stream_plan.rs")] @@ -91,6 +91,10 @@ pub mod health; #[rustfmt::skip] #[path = "sim/telemetry.rs"] pub mod telemetry; + +#[rustfmt::skip] +#[path = "sim/secret.rs"] +pub mod secret; #[rustfmt::skip] #[path = "connector_service.serde.rs"] pub mod connector_service_serde; @@ -158,6 +162,10 @@ pub mod java_binding_serde; #[path = "telemetry.serde.rs"] pub mod telemetry_serde; +#[rustfmt::skip] +#[path = "secret.serde.rs"] +pub mod secret_serde; + #[derive(Clone, PartialEq, Eq, Debug, Error)] #[error("field `{0}` not found")] pub struct PbFieldNotFound(pub &'static str); diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index e4d6b53003788..296f8de4d888f 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -23,7 +23,7 @@ use async_trait::async_trait; use either::Either; use futures::stream::BoxStream; use lru::LruCache; -use risingwave_common::catalog::{CatalogVersion, FunctionId, IndexId, TableId}; +use risingwave_common::catalog::{CatalogVersion, FunctionId, IndexId, SecretId, TableId}; use risingwave_common::config::{MetaConfig, MAX_CONNECTION_WINDOW_SIZE}; use risingwave_common::hash::WorkerSlotMapping; use risingwave_common::system_param::reader::SystemParamsReader; @@ -172,6 +172,25 @@ impl MetaClient { Ok(resp.version) } + pub async fn create_secret( + &self, + secret_name: String, + database_id: u32, + schema_id: u32, + owner_id: u32, + value: Vec, + ) -> Result { + let request = CreateSecretRequest { + name: secret_name, + database_id, + schema_id, + owner_id, + value, + }; + let resp = self.inner.create_secret(request).await?; + Ok(resp.version) + } + pub async fn list_connections(&self, _name: Option<&str>) -> Result> { let request = ListConnectionsRequest {}; let resp = self.inner.list_connections(request).await?; @@ -184,6 +203,14 @@ impl MetaClient { Ok(resp.version) } + pub async fn drop_secret(&self, secret_id: SecretId) -> Result { + let request = DropSecretRequest { + secret_id: secret_id.into(), + }; + let resp = self.inner.drop_secret(request).await?; + Ok(resp.version) + } + /// Register the current node to the cluster and set the corresponding worker id. pub async fn register_new( addr_strategy: MetaAddressStrategy, @@ -1931,12 +1958,14 @@ macro_rules! for_all_meta_rpc { ,{ ddl_client, create_subscription, CreateSubscriptionRequest, CreateSubscriptionResponse } ,{ ddl_client, create_schema, CreateSchemaRequest, CreateSchemaResponse } ,{ ddl_client, create_database, CreateDatabaseRequest, CreateDatabaseResponse } + ,{ ddl_client, create_secret, CreateSecretRequest, CreateSecretResponse } ,{ ddl_client, create_index, CreateIndexRequest, CreateIndexResponse } ,{ ddl_client, create_function, CreateFunctionRequest, CreateFunctionResponse } ,{ ddl_client, drop_table, DropTableRequest, DropTableResponse } ,{ ddl_client, drop_materialized_view, DropMaterializedViewRequest, DropMaterializedViewResponse } ,{ ddl_client, drop_view, DropViewRequest, DropViewResponse } ,{ ddl_client, drop_source, DropSourceRequest, DropSourceResponse } + , {ddl_client, drop_secret, DropSecretRequest, DropSecretResponse} ,{ ddl_client, drop_sink, DropSinkRequest, DropSinkResponse } ,{ ddl_client, drop_subscription, DropSubscriptionRequest, DropSubscriptionResponse } ,{ ddl_client, drop_database, DropDatabaseRequest, DropDatabaseResponse } diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 2c3aa67cfaf36..4b96565a0d683 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -985,6 +985,7 @@ pub enum ShowObject { Subscription { schema: Option }, Columns { table: ObjectName }, Connection { schema: Option }, + Secret { schema: Option }, Function { schema: Option }, Indexes { table: ObjectName }, Cluster, @@ -1033,6 +1034,7 @@ impl fmt::Display for ShowObject { ShowObject::Jobs => write!(f, "JOBS"), ShowObject::ProcessList => write!(f, "PROCESSLIST"), ShowObject::Subscription { schema } => write!(f, "SUBSCRIPTIONS{}", fmt_schema(schema)), + ShowObject::Secret { schema } => write!(f, "SECRETS{}", fmt_schema(schema)), } } } @@ -1109,6 +1111,7 @@ pub struct ExplainOptions { // explain's plan type pub explain_type: ExplainType, } + impl Default for ExplainOptions { fn default() -> Self { Self { @@ -1118,6 +1121,7 @@ impl Default for ExplainOptions { } } } + impl fmt::Display for ExplainOptions { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let default = Self::default(); @@ -1271,6 +1275,9 @@ pub enum Statement { CreateConnection { stmt: CreateConnectionStatement, }, + CreateSecret { + stmt: CreateSecretStatement, + }, /// CREATE FUNCTION /// /// Postgres: @@ -1576,14 +1583,14 @@ impl fmt::Display for Statement { write!(f, "DESCRIBE {}", name)?; Ok(()) } - Statement::ShowObjects{ object: show_object, filter} => { + Statement::ShowObjects { object: show_object, filter } => { write!(f, "SHOW {}", show_object)?; if let Some(filter) = filter { write!(f, " {}", filter)?; } Ok(()) } - Statement::ShowCreateObject{ create_type: show_type, name } => { + Statement::ShowCreateObject { create_type: show_type, name } => { write!(f, "SHOW CREATE {} {}", show_type, name)?; Ok(()) } @@ -1597,7 +1604,7 @@ impl fmt::Display for Statement { source, returning, } => { - write!(f, "INSERT INTO {table_name} ", table_name = table_name,)?; + write!(f, "INSERT INTO {table_name} ", table_name = table_name, )?; if !columns.is_empty() { write!(f, "({}) ", display_comma_separated(columns))?; } @@ -1805,18 +1812,18 @@ impl fmt::Display for Statement { write!(f, "{}", display_comma_separated( include_column_options.iter().map(|option_item: &IncludeOptionItem| { format!("INCLUDE {}{}{}", - option_item.column_type, + option_item.column_type, if let Some(inner_field) = &option_item.inner_field { format!(" {}", inner_field) } else { "".into() } , if let Some(alias) = &option_item.column_alias { - format!(" AS {}", alias) - } else { - "".into() - } - ) + format!(" AS {}", alias) + } else { + "".into() + } + ) }).collect_vec().as_slice() ))?; } @@ -1875,6 +1882,7 @@ impl fmt::Display for Statement { Statement::DeclareCursor { stmt } => write!(f, "DECLARE {}", stmt,), Statement::FetchCursor { stmt } => write!(f, "FETCH {}", stmt), Statement::CloseCursor { stmt } => write!(f, "CLOSE {}", stmt), + Statement::CreateSecret { stmt } => write!(f, "CREATE SECRET {}", stmt), Statement::AlterDatabase { name, operation } => { write!(f, "ALTER DATABASE {} {}", name, operation) } @@ -2003,10 +2011,10 @@ impl fmt::Display for Statement { Ok(()) } Statement::Commit { chain } => { - write!(f, "COMMIT{}", if *chain { " AND CHAIN" } else { "" },) + write!(f, "COMMIT{}", if *chain { " AND CHAIN" } else { "" }, ) } Statement::Rollback { chain } => { - write!(f, "ROLLBACK{}", if *chain { " AND CHAIN" } else { "" },) + write!(f, "ROLLBACK{}", if *chain { " AND CHAIN" } else { "" }, ) } Statement::CreateSchema { schema_name, @@ -2110,7 +2118,7 @@ impl fmt::Display for Statement { Statement::AlterUser(statement) => { write!(f, "ALTER USER {}", statement) } - Statement::AlterSystem{param, value} => { + Statement::AlterSystem { param, value } => { f.write_str("ALTER SYSTEM SET ")?; write!( f, @@ -2528,6 +2536,7 @@ pub enum ObjectType { Database, User, Connection, + Secret, Subscription, } @@ -2543,6 +2552,7 @@ impl fmt::Display for ObjectType { ObjectType::Sink => "SINK", ObjectType::Database => "DATABASE", ObjectType::User => "USER", + ObjectType::Secret => "SECRET", ObjectType::Connection => "CONNECTION", ObjectType::Subscription => "SUBSCRIPTION", }) @@ -2571,11 +2581,13 @@ impl ParseTo for ObjectType { ObjectType::User } else if parser.parse_keyword(Keyword::CONNECTION) { ObjectType::Connection + } else if parser.parse_keyword(Keyword::SECRET) { + ObjectType::Secret } else if parser.parse_keyword(Keyword::SUBSCRIPTION) { ObjectType::Subscription } else { return parser.expected( - "TABLE, VIEW, INDEX, MATERIALIZED VIEW, SOURCE, SINK, SUBSCRIPTION, SCHEMA, DATABASE, USER or CONNECTION after DROP", + "TABLE, VIEW, INDEX, MATERIALIZED VIEW, SOURCE, SINK, SUBSCRIPTION, SCHEMA, DATABASE, USER, SECRET or CONNECTION after DROP", parser.peek_token(), ); }; @@ -2978,6 +2990,7 @@ impl fmt::Display for CreateFunctionBody { Ok(()) } } + #[derive(Debug, Default, Clone, PartialEq, Eq, Hash)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] pub struct CreateFunctionWithOptions { diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index aef8ec417f605..2e5b281d1938f 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -13,6 +13,7 @@ // limitations under the License. use core::fmt; +use core::fmt::Formatter; use std::fmt::Write; use itertools::Itertools; @@ -21,7 +22,7 @@ use serde::{Deserialize, Serialize}; use super::ddl::SourceWatermark; use super::legacy_source::{parse_source_schema, CompatibleSourceSchema}; -use super::{EmitMode, Ident, ObjectType, Query}; +use super::{EmitMode, Ident, ObjectType, Query, Value}; use crate::ast::{ display_comma_separated, display_separated, ColumnDef, ObjectName, SqlOption, TableConstraint, }; @@ -850,6 +851,47 @@ impl fmt::Display for CreateConnectionStatement { } } +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub struct CreateSecretStatement { + pub if_not_exists: bool, + pub secret_name: ObjectName, + pub credential: Value, + pub with_properties: WithProperties, +} + +impl ParseTo for CreateSecretStatement { + fn parse_to(parser: &mut Parser) -> Result { + impl_parse_to!(if_not_exists => [Keyword::IF, Keyword::NOT, Keyword::EXISTS], parser); + impl_parse_to!(secret_name: ObjectName, parser); + impl_parse_to!(with_properties: WithProperties, parser); + let mut credential = Value::Null; + if parser.parse_keyword(Keyword::AS) { + credential = parser.parse_value()?; + } + Ok(Self { + if_not_exists, + secret_name, + credential, + with_properties, + }) + } +} + +impl fmt::Display for CreateSecretStatement { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + let mut v: Vec = vec![]; + impl_fmt_display!(if_not_exists => [Keyword::IF, Keyword::NOT, Keyword::EXISTS], v, self); + impl_fmt_display!(secret_name, v, self); + impl_fmt_display!(with_properties, v, self); + if self.credential != Value::Null { + v.push("AS".to_string()); + impl_fmt_display!(credential, v, self); + } + v.iter().join(" ").fmt(f) + } +} + #[derive(Debug, Clone, PartialEq, Eq, Hash)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] pub struct AstVec(pub Vec); diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index e3f968b0caac6..0ed69adab3ad0 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -448,6 +448,8 @@ define_keywords!( SCROLL, SEARCH, SECOND, + SECRET, + SECRETS, SELECT, SENSITIVE, SEQUENCE, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index a152e9e8d90da..243d9e695ff3a 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -2178,6 +2178,8 @@ impl Parser { self.parse_create_database() } else if self.parse_keyword(Keyword::USER) { self.parse_create_user() + } else if self.parse_keyword(Keyword::SECRET) { + self.parse_create_secret() } else { self.expected("an object type after CREATE", self.peek_token()) } @@ -2529,6 +2531,12 @@ impl Parser { Ok(Statement::CreateUser(CreateUserStatement::parse_to(self)?)) } + fn parse_create_secret(&mut self) -> Result { + Ok(Statement::CreateSecret { + stmt: CreateSecretStatement::parse_to(self)?, + }) + } + pub fn parse_with_properties(&mut self) -> Result, ParserError> { Ok(self .parse_options_with_preceding_keyword(Keyword::WITH)? @@ -3721,7 +3729,7 @@ impl Parser { } /// Parse a literal value (numbers, strings, date/time, booleans) - fn parse_value(&mut self) -> Result { + pub fn parse_value(&mut self) -> Result { let token = self.next_token(); match token.token { Token::Word(w) => match w.keyword { @@ -4596,6 +4604,14 @@ impl Parser { return self.expected("from after columns", self.peek_token()); } } + Keyword::SECRETS => { + return Ok(Statement::ShowObjects { + object: ShowObject::Secret { + schema: self.parse_from_and_identifier()?, + }, + filter: self.parse_show_statement_filter()?, + }); + } Keyword::CONNECTIONS => { return Ok(Statement::ShowObjects { object: ShowObject::Connection { diff --git a/src/sqlparser/tests/testdata/create.yaml b/src/sqlparser/tests/testdata/create.yaml index 670e325ea08e8..831886b9bdb36 100644 --- a/src/sqlparser/tests/testdata/create.yaml +++ b/src/sqlparser/tests/testdata/create.yaml @@ -116,3 +116,7 @@ error_msg: |- sql parser error: Expected literal string, found: null at line:1, column:45 Near " tmp with encrypted password null" +- input: CREATE SECRET secret1 WITH (backend = 'meta') AS 'demo-secret' + formatted_sql: CREATE SECRET secret1 WITH (backend = 'meta') AS 'demo-secret' +- input: CREATE SECRET IF NOT EXISTS secret2 WITH (backend = 'meta') AS 'demo-secret + error_msg: 'sql parser error: Unterminated string literal at Line: 1, Column 62' diff --git a/src/sqlparser/tests/testdata/drop.yaml b/src/sqlparser/tests/testdata/drop.yaml index 3fd366d3ea71d..6b8a70d0bf9be 100644 --- a/src/sqlparser/tests/testdata/drop.yaml +++ b/src/sqlparser/tests/testdata/drop.yaml @@ -16,3 +16,7 @@ formatted_sql: DROP USER user - input: DROP USER IF EXISTS user formatted_sql: DROP USER IF EXISTS user +- input: DROP SECRET secret + formatted_sql: DROP SECRET secret +- input: DROP SECRET IF EXISTS secret + formatted_sql: DROP SECRET IF EXISTS secret diff --git a/src/sqlparser/tests/testdata/show.yaml b/src/sqlparser/tests/testdata/show.yaml index 8f15d2667c589..a3df33bff2cbc 100644 --- a/src/sqlparser/tests/testdata/show.yaml +++ b/src/sqlparser/tests/testdata/show.yaml @@ -53,3 +53,9 @@ - input: SHOW INDEXES FROM t formatted_sql: SHOW INDEXES FROM t formatted_ast: 'ShowObjects { object: Indexes { table: ObjectName([Ident { value: "t", quote_style: None }]) }, filter: None }' +- input: SHOW SECRETS + formatted_sql: SHOW SECRETS + formatted_ast: 'ShowObjects { object: Secret { schema: None }, filter: None }' +- input: SHOW SECRETS FROM t + formatted_sql: SHOW SECRETS FROM t + formatted_ast: 'ShowObjects { object: Secret { schema: Some(Ident { value: "t", quote_style: None }) }, filter: None }' diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index 8764c2bc91456..4f55c524942bc 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -26,6 +26,7 @@ use crate::types::Row; pub type RowSet = Vec; pub type RowSetResult = Result; + pub trait ValuesStream = Stream + Unpin + Send; #[derive(Copy, Clone, Debug, Eq, PartialEq)] @@ -56,6 +57,7 @@ pub enum StatementType { CREATE_AGGREGATE, CREATE_FUNCTION, CREATE_CONNECTION, + CREATE_SECRET, COMMENT, DECLARE_CURSOR, DESCRIBE, @@ -74,6 +76,7 @@ pub enum StatementType { DROP_DATABASE, DROP_USER, DROP_CONNECTION, + DROP_SECRET, ALTER_DATABASE, ALTER_SCHEMA, ALTER_INDEX, @@ -118,6 +121,7 @@ impl std::fmt::Display for StatementType { } pub trait Callback = Future> + Send; + pub type BoxedCallback = Pin>; pub struct PgResponse { @@ -293,6 +297,7 @@ impl StatementType { risingwave_sqlparser::ast::ObjectType::Connection => { Ok(StatementType::DROP_CONNECTION) } + risingwave_sqlparser::ast::ObjectType::Secret => Ok(StatementType::DROP_SECRET), risingwave_sqlparser::ast::ObjectType::Subscription => { Ok(StatementType::DROP_SUBSCRIPTION) }