From cd787f6d8ea4ee39d1b5552fbaa9e894b443dc61 Mon Sep 17 00:00:00 2001 From: Zihao Xu Date: Tue, 9 Jan 2024 13:03:04 -0500 Subject: [PATCH 01/17] chore(sql-udf): update test cases for implicit type cast support (#14458) --- e2e_test/udf/sql_udf.slt | 28 +++++++++++++++++----------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/e2e_test/udf/sql_udf.slt b/e2e_test/udf/sql_udf.slt index 8b89010f70a93..e1100834c9bbd 100644 --- a/e2e_test/udf/sql_udf.slt +++ b/e2e_test/udf/sql_udf.slt @@ -111,30 +111,33 @@ create function add_error(INT, INT) returns int language sql as $$select $1 + $2 statement ok create function add_sub(INT, FLOAT, INT) returns float language sql as $$select -$1 + $2 - $3$$; +# Complex types interleaving +statement ok +create function add_sub_types(INT, BIGINT, FLOAT, DECIMAL, REAL) returns real language sql as 'select $1 + $2 - $3 + $4 + $5'; + statement ok create function add_sub_return(INT, FLOAT, INT) returns float language sql return -$1 + $2 - $3; -# Note: need EXPLICIT type cast in order to call the multiple types interleaving sql udf query I -select add_sub(1::INT, 5.1415926::FLOAT, 1::INT); +select add_sub(1, 5.1415926, 1); ---- 3.1415926 -# Without EXPLICIT type cast -statement error unsupported function: "add_sub" -select add_sub(1, 3.14, 2); - -# Same as above, need EXPLICIT type cast to make the binding works query I -select add_sub_return(1::INT, 5.1415926::FLOAT, 1::INT); +select add_sub_return(1, 5.1415926, 1); ---- 3.1415926 query III -select add(1, -1), sub(1, 1), add_sub(1::INT, 5.1415926::FLOAT, 1::INT); +select add(1, -1), sub(1, 1), add_sub(1, 5.1415926, 1); ---- 0 0 3.1415926 +query I +select add_sub_types(1, 1919810114514, 3.1415926, 1.123123, 101010.191919); +---- +1919810215523.1734494 + # Create another mock table statement ok create table t2 (c1 INT, c2 FLOAT, c3 INT); @@ -143,14 +146,14 @@ statement ok insert into t2 values (1, 3.14, 2), (2, 4.44, 5), (20, 10.30, 02); query IIIIII -select c1, c2, c3, add(c1, c3), sub(c1, c3), add_sub(c1::INT, c2::FLOAT, c3::INT) from t2 order by c1 asc; +select c1, c2, c3, add(c1, c3), sub(c1, c3), add_sub(c1, c2, c3) from t2 order by c1 asc; ---- 1 3.14 2 3 -1 0.14000000000000012 2 4.44 5 7 -3 -2.5599999999999996 20 10.3 2 22 18 -11.7 query IIIIII -select c1, c2, c3, add(c1, c3), sub(c1, c3), add_sub_return(c1::INT, c2::FLOAT, c3::INT) from t2 order by c1 asc; +select c1, c2, c3, add(c1, c3), sub(c1, c3), add_sub_return(c1, c2, c3) from t2 order by c1 asc; ---- 1 3.14 2 3 -1 0.14000000000000012 2 4.44 5 7 -3 -2.5599999999999996 @@ -184,6 +187,9 @@ drop function call_regexp_replace; statement ok drop function add_sub_wrapper; +statement ok +drop function add_sub_types; + # Drop the mock table statement ok drop table t1; From ead133d80bd1a58bebbaa4b0edac63ab8a5c08de Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 9 Jan 2024 17:38:27 -0500 Subject: [PATCH 02/17] fix(test): fix the wrong suffix in cron test (#14467) --- ci/workflows/main-cron.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 56d4695beafbc..341ea49585558 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -508,7 +508,7 @@ steps: - label: "S3 source on OpenDAL fs engine" key: "s3-source-test-for-opendal-fs-engine" - command: "ci/scripts/s3-source-test-for-opendal-fs-engine.sh -p ci-release -s run.csv" + command: "ci/scripts/s3-source-test-for-opendal-fs-engine.sh -p ci-release -s run.py" if: | !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-s3-source-tests" @@ -531,7 +531,7 @@ steps: # TODO(Kexiang): Enable this test after we have a GCS_SOURCE_TEST_CONF. # - label: "GCS source on OpenDAL fs engine" # key: "s3-source-test-for-opendal-fs-engine" - # command: "ci/scripts/s3-source-test-for-opendal-fs-engine.sh -p ci-release -s gcs.csv" + # command: "ci/scripts/s3-source-test-for-opendal-fs-engine.sh -p ci-release -s gcs_source.py" # if: | # !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null # || build.pull_request.labels includes "ci/run-s3-source-tests" From 19bd2daab424aa29c1faf70555b74233bcbd0ab3 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Wed, 10 Jan 2024 09:47:12 +0800 Subject: [PATCH 03/17] chore: pack telnet kcat in release image (#14432) --- docker/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/Dockerfile b/docker/Dockerfile index 231812f83e8d1..58e93de54bcef 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -118,7 +118,7 @@ LABEL org.opencontainers.image.source https://github.com/risingwavelabs/risingwa RUN apt-get update && apt-get -y install linux-tools-generic \ && ln -s "$(find /usr/lib/linux-tools/*/perf | head -1)" /usr/local/bin/perf -RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get -y install gdb libpam-krb5 krb5-user \ +RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get -y install gdb libpam-krb5 krb5-user telnet kafkacat \ && rm -rf /var/lib/{apt,dpkg,cache,log}/ RUN mkdir -p /risingwave/bin/connector-node && mkdir -p /risingwave/lib From f3b0650cc30273e561b81f383221a708a092e6ea Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Wed, 10 Jan 2024 09:58:18 +0800 Subject: [PATCH 04/17] feat(sql-backend): integrate HummockManager with model V2 (#14355) --- .../migration/src/m20231008_020431_hummock.rs | 2 +- src/meta/src/backup_restore/backup_manager.rs | 9 +- src/meta/src/hummock/compactor_manager.rs | 15 +- src/meta/src/hummock/error.rs | 7 + .../manager/compaction_group_manager.rs | 232 ++++++---- src/meta/src/hummock/manager/context.rs | 23 +- src/meta/src/hummock/manager/gc.rs | 21 +- src/meta/src/hummock/manager/mod.rs | 412 +++++++++++------- src/meta/src/hummock/manager/sequence.rs | 112 +++-- src/meta/src/hummock/manager/tests.rs | 60 ++- src/meta/src/hummock/manager/utils.rs | 64 +++ src/meta/src/hummock/manager/versioning.rs | 18 +- src/meta/src/hummock/model/ext/hummock.rs | 35 +- src/meta/src/manager/env.rs | 11 + src/meta/src/model/mod.rs | 259 +++++++++++ 15 files changed, 940 insertions(+), 340 deletions(-) create mode 100644 src/meta/src/hummock/manager/utils.rs diff --git a/src/meta/model_v2/migration/src/m20231008_020431_hummock.rs b/src/meta/model_v2/migration/src/m20231008_020431_hummock.rs index f65c396eba3b8..5f1f26b3e21db 100644 --- a/src/meta/model_v2/migration/src/m20231008_020431_hummock.rs +++ b/src/meta/model_v2/migration/src/m20231008_020431_hummock.rs @@ -150,7 +150,7 @@ impl MigrationTrait for Migration { .boolean() .not_null(), ) - .col(ColumnDef::new(HummockVersionDelta::FullVersionDelta).binary()) + .col(ColumnDef::new(HummockVersionDelta::FullVersionDelta).json_binary()) .to_owned(), ) .await?; diff --git a/src/meta/src/backup_restore/backup_manager.rs b/src/meta/src/backup_restore/backup_manager.rs index 725c9a699ee4e..a029795617f9c 100644 --- a/src/meta/src/backup_restore/backup_manager.rs +++ b/src/meta/src/backup_restore/backup_manager.rs @@ -31,8 +31,9 @@ use tokio::task::JoinHandle; use crate::backup_restore::meta_snapshot_builder; use crate::backup_restore::metrics::BackupManagerMetrics; +use crate::hummock::sequence::next_meta_backup_id; use crate::hummock::{HummockManagerRef, HummockVersionSafePoint}; -use crate::manager::{IdCategory, LocalNotification, MetaSrvEnv}; +use crate::manager::{LocalNotification, MetaSrvEnv}; use crate::rpc::metrics::MetaMetrics; use crate::MetaResult; @@ -215,11 +216,7 @@ impl BackupManager { )) } - let job_id = self - .env - .id_gen_manager() - .generate::<{ IdCategory::Backup }>() - .await?; + let job_id = next_meta_backup_id(&self.env).await?; self.latest_job_info .store(Arc::new((job_id, BackupJobStatus::Running, "".into()))); let hummock_version_safe_point = self.hummock_manager.register_safe_point().await; diff --git a/src/meta/src/hummock/compactor_manager.rs b/src/meta/src/hummock/compactor_manager.rs index 7784a8637fb8e..71cba056ca535 100644 --- a/src/meta/src/hummock/compactor_manager.rs +++ b/src/meta/src/hummock/compactor_manager.rs @@ -28,7 +28,7 @@ use risingwave_pb::hummock::{ use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use crate::manager::MetaSrvEnv; -use crate::model::MetadataModel; +use crate::model::{MetadataModel, MetadataModelError}; use crate::MetaResult; pub type CompactorManagerRef = Arc; @@ -125,8 +125,19 @@ pub struct CompactorManagerInner { impl CompactorManagerInner { pub async fn with_meta(env: MetaSrvEnv) -> MetaResult { + use risingwave_meta_model_v2::compaction_task; + use sea_orm::EntityTrait; // Retrieve the existing task assignments from metastore. - let task_assignment = CompactTaskAssignment::list(env.meta_store()).await?; + let task_assignment: Vec = match env.sql_meta_store() { + None => CompactTaskAssignment::list(env.meta_store()).await?, + Some(sql_meta_store) => compaction_task::Entity::find() + .all(&sql_meta_store.conn) + .await + .map_err(MetadataModelError::from)? + .into_iter() + .map(Into::into) + .collect(), + }; let mut manager = Self { task_expiry_seconds: env.opts.compaction_task_max_heartbeat_interval_secs, task_heartbeats: Default::default(), diff --git a/src/meta/src/hummock/error.rs b/src/meta/src/hummock/error.rs index f16d32289dc03..434de47623310 100644 --- a/src/meta/src/hummock/error.rs +++ b/src/meta/src/hummock/error.rs @@ -15,6 +15,7 @@ use risingwave_hummock_sdk::{HummockContextId, HummockSstableObjectId}; use risingwave_object_store::object::ObjectError; use risingwave_rpc_client::error::ToTonicStatus; +use sea_orm::DbErr; use thiserror::Error; use crate::model::MetadataModelError; @@ -83,6 +84,12 @@ impl From for Error { } } +impl From for Error { + fn from(value: DbErr) -> Self { + MetadataModelError::from(value).into() + } +} + impl From for tonic::Status { fn from(err: Error) -> Self { err.to_status(tonic::Code::Internal, "hummock") diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index b33e3ef8b2e8e..ab569f5ed468e 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -25,6 +25,7 @@ use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ }; use risingwave_hummock_sdk::compaction_group::{StateTableId, StaticCompactionGroupId}; use risingwave_hummock_sdk::CompactionGroupId; +use risingwave_meta_model_v2::compaction_config; use risingwave_pb::hummock::compact_task::TaskStatus; use risingwave_pb::hummock::group_delta::DeltaType; use risingwave_pb::hummock::hummock_version_delta::GroupDeltas; @@ -37,18 +38,23 @@ use tokio::sync::{OnceCell, RwLock}; use tracing::warn; use super::write_lock; +use crate::controller::SqlMetaStore; use crate::hummock::compaction::compaction_config::{ validate_compaction_config, CompactionConfigBuilder, }; use crate::hummock::error::{Error, Result}; -use crate::hummock::manager::{drop_sst, read_lock, HummockManager}; +use crate::hummock::manager::{ + commit_multi_var, create_trx_wrapper, drop_sst, read_lock, HummockManager, +}; use crate::hummock::metrics_utils::remove_compaction_group_in_sst_stat; use crate::hummock::model::CompactionGroup; -use crate::manager::{IdCategory, MetaSrvEnv}; +use crate::hummock::sequence::{next_compaction_group_id, next_sstable_object_id}; +use crate::manager::MetaSrvEnv; use crate::model::{ - BTreeMapEntryTransaction, BTreeMapTransaction, MetadataModel, TableFragments, ValTransaction, + BTreeMapEntryTransaction, BTreeMapEntryTransactionWrapper, BTreeMapTransaction, + BTreeMapTransactionWrapper, MetadataModel, MetadataModelError, TableFragments, ValTransaction, }; -use crate::storage::{MetaStore, Transaction}; +use crate::storage::MetaStore; impl HummockManager { pub(super) async fn build_compaction_group_manager( @@ -68,6 +74,7 @@ impl HummockManager { let compaction_group_manager = RwLock::new(CompactionGroupManager { compaction_groups: BTreeMap::new(), default_config, + sql_meta_store: env.sql_meta_store(), }); compaction_group_manager .write() @@ -188,10 +195,14 @@ impl HummockManager { } // All NewCompactionGroup pairs are mapped to one new compaction group. let new_compaction_group_id: OnceCell = OnceCell::new(); - let mut new_version_delta = BTreeMapEntryTransaction::new_insert( - &mut versioning.hummock_version_deltas, - current_version.id + 1, - build_version_delta_after_version(current_version), + let mut new_version_delta = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapEntryTransactionWrapper, + BTreeMapEntryTransaction::new_insert( + &mut versioning.hummock_version_deltas, + current_version.id + 1, + build_version_delta_after_version(current_version), + ) ); for (table_id, raw_group_id) in pairs { @@ -200,9 +211,7 @@ impl HummockManager { let mut is_group_init = false; group_id = *new_compaction_group_id .get_or_try_init(|| async { - self.env - .id_gen_manager() - .generate::<{ IdCategory::CompactionGroup }>() + next_compaction_group_id(&self.env) .await .map(|new_group_id| { is_group_init = true; @@ -249,12 +258,12 @@ impl HummockManager { let mut current_version = versioning.current_version.clone(); let sst_split_info = current_version.apply_version_delta(&new_version_delta); assert!(sst_split_info.is_empty()); - - let mut trx = Transaction::default(); - new_version_delta.apply_to_txn(&mut trx).await?; - self.env.meta_store().txn(trx).await?; + commit_multi_var!( + self.env.meta_store(), + self.sql_meta_store(), + new_version_delta + )?; versioning.current_version = current_version; - new_version_delta.commit(); self.notify_last_version_delta(versioning); @@ -269,13 +278,15 @@ impl HummockManager { let mut versioning_guard = write_lock!(self, versioning).await; let versioning = versioning_guard.deref_mut(); let current_version = &versioning.current_version; - - let mut new_version_delta = BTreeMapEntryTransaction::new_insert( - &mut versioning.hummock_version_deltas, - current_version.id + 1, - build_version_delta_after_version(current_version), + let mut new_version_delta = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapEntryTransactionWrapper, + BTreeMapEntryTransaction::new_insert( + &mut versioning.hummock_version_deltas, + current_version.id + 1, + build_version_delta_after_version(current_version), + ) ); - let mut modified_groups: HashMap = HashMap::new(); // Remove member tables @@ -312,7 +323,11 @@ impl HummockManager { } // Remove empty group, GC SSTs and remove metric. - let mut branched_ssts = BTreeMapTransaction::new(&mut versioning.branched_ssts); + let mut branched_ssts = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning.branched_ssts) + ); let groups_to_remove = modified_groups .into_iter() .filter_map(|(group_id, member_count)| { @@ -344,10 +359,12 @@ impl HummockManager { let mut current_version = versioning.current_version.clone(); let sst_split_info = current_version.apply_version_delta(&new_version_delta); assert!(sst_split_info.is_empty()); + commit_multi_var!( + self.env.meta_store(), + self.sql_meta_store(), + new_version_delta + )?; - let mut trx = Transaction::default(); - new_version_delta.apply_to_txn(&mut trx).await?; - self.env.meta_store().txn(trx).await?; for group_id in &groups_to_remove { let max_level = versioning .current_version @@ -356,9 +373,7 @@ impl HummockManager { .len(); remove_compaction_group_in_sst_stat(&self.metrics, *group_id, max_level); } - versioning.current_version = current_version; - new_version_delta.commit(); branched_ssts.commit_memory(); self.notify_last_version_delta(versioning); @@ -506,22 +521,23 @@ impl HummockManager { ))); } } - - let mut new_version_delta = BTreeMapEntryTransaction::new_insert( - &mut versioning.hummock_version_deltas, - current_version.id + 1, - build_version_delta_after_version(current_version), - ); - let new_sst_start_id = self - .env - .id_gen_manager() - .generate_interval::<{ IdCategory::HummockSstableId }>( - current_version.count_new_ssts_in_group_split( - parent_group_id, - HashSet::from_iter(table_ids.clone()), - ), + let mut new_version_delta = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapEntryTransactionWrapper, + BTreeMapEntryTransaction::new_insert( + &mut versioning.hummock_version_deltas, + current_version.id + 1, + build_version_delta_after_version(current_version), ) - .await?; + ); + let new_sst_start_id = next_sstable_object_id( + &self.env, + current_version.count_new_ssts_in_group_split( + parent_group_id, + HashSet::from_iter(table_ids.clone()), + ), + ) + .await?; let mut new_group = None; let target_compaction_group_id = match target_group_id { Some(compaction_group_id) => { @@ -561,11 +577,7 @@ impl HummockManager { } None => { // All NewCompactionGroup pairs are mapped to one new compaction group. - let new_compaction_group_id = self - .env - .id_gen_manager() - .generate::<{ IdCategory::CompactionGroup }>() - .await?; + let new_compaction_group_id = next_compaction_group_id(&self.env).await?; // The new config will be persisted later. let mut config = self .compaction_group_manager @@ -609,29 +621,42 @@ impl HummockManager { let mut current_version = versioning.current_version.clone(); let sst_split_info = current_version.apply_version_delta(&new_version_delta); - let mut branched_ssts = BTreeMapTransaction::<'_, _, _>::new(&mut versioning.branched_ssts); - let mut trx = Transaction::default(); - new_version_delta.apply_to_txn(&mut trx).await?; + // `branched_ssts` only commit in memory, so `TXN` make no difference. + let mut branched_ssts = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning.branched_ssts) + ); if let Some((new_compaction_group_id, config)) = new_group { let mut compaction_group_manager = self.compaction_group_manager.write().await; - let insert = BTreeMapEntryTransaction::new_insert( - &mut compaction_group_manager.compaction_groups, - new_compaction_group_id, - CompactionGroup { - group_id: new_compaction_group_id, - compaction_config: Arc::new(config), - }, + let insert = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapEntryTransactionWrapper, + BTreeMapEntryTransaction::new_insert( + &mut compaction_group_manager.compaction_groups, + new_compaction_group_id, + CompactionGroup { + group_id: new_compaction_group_id, + compaction_config: Arc::new(config), + }, + ) ); - insert.apply_to_txn(&mut trx).await?; - self.env.meta_store().txn(trx).await?; - insert.commit(); - + commit_multi_var!( + self.env.meta_store(), + self.sql_meta_store(), + new_version_delta, + insert + )?; // Currently, only splitting out a single table_id is supported. for table_id in table_ids { table_to_partition.insert(table_id, partition_vnode_count); } } else { - self.env.meta_store().txn(trx).await?; + commit_multi_var!( + self.env.meta_store(), + self.sql_meta_store(), + new_version_delta + )?; } versioning.current_version = current_version; // Updates SST split info @@ -650,7 +675,6 @@ impl HummockManager { } } } - new_version_delta.commit(); branched_ssts.commit_memory(); self.notify_last_version_delta(versioning); drop(versioning_guard); @@ -748,20 +772,32 @@ impl HummockManager { /// 1. initialize default static compaction group. /// 2. register new table to new compaction group. /// 3. move existent table to new compaction group. -#[derive(Default)] pub(super) struct CompactionGroupManager { compaction_groups: BTreeMap, default_config: CompactionConfig, + sql_meta_store: Option, } impl CompactionGroupManager { async fn init(&mut self, meta_store: &S) -> Result<()> { let loaded_compaction_groups: BTreeMap = - CompactionGroup::list(meta_store) - .await? - .into_iter() - .map(|cg| (cg.group_id(), cg)) - .collect(); + match &self.sql_meta_store { + None => CompactionGroup::list(meta_store) + .await? + .into_iter() + .map(|cg| (cg.group_id(), cg)) + .collect(), + Some(sql_meta_store) => { + use sea_orm::EntityTrait; + compaction_config::Entity::find() + .all(&sql_meta_store.conn) + .await + .map_err(MetadataModelError::from)? + .into_iter() + .map(|m| (m.compaction_group_id as CompactionGroupId, m.into())) + .collect() + } + }; if !loaded_compaction_groups.is_empty() { self.compaction_groups = loaded_compaction_groups; } @@ -786,7 +822,11 @@ impl CompactionGroupManager { compaction_group_ids: &[CompactionGroupId], meta_store: &S, ) -> Result> { - let mut compaction_groups = BTreeMapTransaction::new(&mut self.compaction_groups); + let mut compaction_groups = create_trx_wrapper!( + self.sql_meta_store, + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut self.compaction_groups,) + ); for id in compaction_group_ids { if compaction_groups.contains_key(id) { continue; @@ -794,10 +834,8 @@ impl CompactionGroupManager { let new_entry = CompactionGroup::new(*id, self.default_config.clone()); compaction_groups.insert(*id, new_entry); } - let mut trx = Transaction::default(); - compaction_groups.apply_to_txn(&mut trx).await?; - meta_store.txn(trx).await?; - compaction_groups.commit(); + commit_multi_var!(meta_store, self.sql_meta_store, compaction_groups)?; + let r = compaction_group_ids .iter() .map(|id| (*id, self.compaction_groups[id].clone())) @@ -823,7 +861,11 @@ impl CompactionGroupManager { config_to_update: &[MutableConfig], meta_store: &S, ) -> Result> { - let mut compaction_groups = BTreeMapTransaction::new(&mut self.compaction_groups); + let mut compaction_groups = create_trx_wrapper!( + self.sql_meta_store, + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut self.compaction_groups,) + ); let mut result = Vec::with_capacity(compaction_group_ids.len()); for compaction_group_id in compaction_group_ids.iter().unique() { let group = compaction_groups.get(compaction_group_id).ok_or_else(|| { @@ -839,11 +881,7 @@ impl CompactionGroupManager { compaction_groups.insert(*compaction_group_id, new_group.clone()); result.push(new_group); } - - let mut trx = Transaction::default(); - compaction_groups.apply_to_txn(&mut trx).await?; - meta_store.txn(trx).await?; - compaction_groups.commit(); + commit_multi_var!(meta_store, self.sql_meta_store, compaction_groups)?; Ok(result) } @@ -855,18 +893,19 @@ impl CompactionGroupManager { config: CompactionConfig, meta_store: &S, ) -> Result<()> { - let insert = BTreeMapEntryTransaction::new_insert( - &mut self.compaction_groups, - group_id, - CompactionGroup { + let insert = create_trx_wrapper!( + self.sql_meta_store, + BTreeMapEntryTransactionWrapper, + BTreeMapEntryTransaction::new_insert( + &mut self.compaction_groups, group_id, - compaction_config: Arc::new(config), - }, + CompactionGroup { + group_id, + compaction_config: Arc::new(config), + }, + ) ); - let mut trx = Transaction::default(); - insert.apply_to_txn(&mut trx).await?; - meta_store.txn(trx).await?; - insert.commit(); + commit_multi_var!(meta_store, self.sql_meta_store, insert)?; Ok(()) } @@ -876,7 +915,11 @@ impl CompactionGroupManager { existing_groups: HashSet, meta_store: &S, ) -> Result<()> { - let mut compaction_groups = BTreeMapTransaction::new(&mut self.compaction_groups); + let mut compaction_groups = create_trx_wrapper!( + self.sql_meta_store, + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut self.compaction_groups,) + ); let stale_group = compaction_groups .tree_ref() .keys() @@ -889,10 +932,7 @@ impl CompactionGroupManager { for group in stale_group { compaction_groups.remove(group); } - let mut trx = Transaction::default(); - compaction_groups.apply_to_txn(&mut trx).await?; - meta_store.txn(trx).await?; - compaction_groups.commit(); + commit_multi_var!(meta_store, self.sql_meta_store, compaction_groups)?; Ok(()) } } diff --git a/src/meta/src/hummock/manager/context.rs b/src/meta/src/hummock/manager/context.rs index 1cf3e580ed44c..98588579137bd 100644 --- a/src/meta/src/hummock/manager/context.rs +++ b/src/meta/src/hummock/manager/context.rs @@ -26,12 +26,12 @@ use risingwave_pb::hummock::ValidationTask; use crate::hummock::error::{Error, Result}; use crate::hummock::manager::{ - commit_multi_var, read_lock, start_measure_real_process_timer, write_lock, + commit_multi_var, create_trx_wrapper, read_lock, start_measure_real_process_timer, write_lock, }; use crate::hummock::HummockManager; use crate::manager::META_NODE_ID; -use crate::model::{BTreeMapTransaction, ValTransaction}; -use crate::storage::Transaction; +use crate::model::{BTreeMapTransaction, BTreeMapTransactionWrapper, ValTransaction}; +use crate::storage::MetaStore; impl HummockManager { /// Release resources pinned by these contexts, including: @@ -51,16 +51,23 @@ impl HummockManager { let mut versioning_guard = write_lock!(self, versioning).await; let versioning = versioning_guard.deref_mut(); - let mut pinned_versions = BTreeMapTransaction::new(&mut versioning.pinned_versions); - let mut pinned_snapshots = BTreeMapTransaction::new(&mut versioning.pinned_snapshots); + let mut pinned_versions = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning.pinned_versions,) + ); + let mut pinned_snapshots = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning.pinned_snapshots,) + ); for context_id in context_ids.as_ref() { pinned_versions.remove(*context_id); pinned_snapshots.remove(*context_id); } commit_multi_var!( - self, - None, - Transaction::default(), + self.env.meta_store(), + self.sql_meta_store(), pinned_versions, pinned_snapshots )?; diff --git a/src/meta/src/hummock/manager/gc.rs b/src/meta/src/hummock/manager/gc.rs index b4506ac30974f..e17f90aaad170 100644 --- a/src/meta/src/hummock/manager/gc.rs +++ b/src/meta/src/hummock/manager/gc.rs @@ -26,11 +26,13 @@ use risingwave_pb::common::WorkerType; use risingwave_pb::hummock::FullScanTask; use crate::hummock::error::{Error, Result}; -use crate::hummock::manager::{commit_multi_var, read_lock, write_lock, ResponseEvent}; +use crate::hummock::manager::{ + commit_multi_var, create_trx_wrapper, read_lock, write_lock, ResponseEvent, +}; use crate::hummock::HummockManager; use crate::manager::MetadataManager; -use crate::model::{BTreeMapTransaction, ValTransaction}; -use crate::storage::Transaction; +use crate::model::{BTreeMapTransaction, BTreeMapTransactionWrapper, ValTransaction}; +use crate::storage::MetaStore; impl HummockManager { /// Gets SST objects that is safe to be deleted from object store. @@ -79,8 +81,11 @@ impl HummockManager { if !versioning.version_safe_points.is_empty() { return Ok((0, deltas_to_delete.len())); } - let mut hummock_version_deltas = - BTreeMapTransaction::new(&mut versioning.hummock_version_deltas); + let mut hummock_version_deltas = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning.hummock_version_deltas,) + ); let batch = deltas_to_delete .iter() .take(batch_size) @@ -92,7 +97,11 @@ impl HummockManager { for delta_id in &batch { hummock_version_deltas.remove(*delta_id); } - commit_multi_var!(self, None, Transaction::default(), hummock_version_deltas)?; + commit_multi_var!( + self.env.meta_store(), + self.sql_meta_store(), + hummock_version_deltas + )?; #[cfg(test)] { drop(versioning_guard); diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index b355a18bcb8d4..c4059efce8273 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -45,6 +45,10 @@ use risingwave_hummock_sdk::{ HummockContextId, HummockEpoch, HummockSstableId, HummockSstableObjectId, HummockVersionId, SstObjectIdRange, INVALID_VERSION_ID, }; +use risingwave_meta_model_v2::{ + compaction_status, compaction_task, hummock_pinned_snapshot, hummock_pinned_version, + hummock_version_delta, hummock_version_stats, +}; use risingwave_pb::hummock::compact_task::{self, TaskStatus, TaskType}; use risingwave_pb::hummock::group_delta::DeltaType; use risingwave_pb::hummock::rise_ctl_update_compaction_config_request::mutable_config; @@ -80,16 +84,18 @@ use crate::hummock::metrics_utils::{ trigger_mv_stat, trigger_pin_unpin_snapshot_state, trigger_pin_unpin_version_state, trigger_split_stat, trigger_sst_stat, trigger_version_stat, trigger_write_stop_stats, }; +use crate::hummock::sequence::next_compaction_task_id; use crate::hummock::{CompactorManagerRef, TASK_NORMAL}; #[cfg(any(test, feature = "test"))] use crate::manager::{ClusterManagerRef, FragmentManagerRef}; -use crate::manager::{IdCategory, MetaSrvEnv, MetadataManager, TableId, META_NODE_ID}; +use crate::manager::{MetaSrvEnv, MetadataManager, TableId, META_NODE_ID}; use crate::model::{ - BTreeMapEntryTransaction, BTreeMapTransaction, ClusterId, MetadataModel, ValTransaction, - VarTransaction, + BTreeMapEntryTransaction, BTreeMapEntryTransactionWrapper, BTreeMapTransaction, + BTreeMapTransactionWrapper, ClusterId, MetadataModel, MetadataModelError, ValTransaction, + VarTransaction, VarTransactionWrapper, }; use crate::rpc::metrics::MetaMetrics; -use crate::storage::{MetaStore, MetaStoreRef, Transaction}; +use crate::storage::MetaStore; mod compaction_group_manager; mod context; @@ -101,10 +107,12 @@ pub use versioning::HummockVersionSafePoint; use versioning::*; pub(crate) mod checkpoint; mod compaction; -mod sequence; +pub mod sequence; +mod utils; mod worker; use compaction::*; +pub(crate) use utils::*; type Snapshot = ArcSwap; const HISTORY_TABLE_INFO_STATISTIC_TIME: usize = 240; @@ -162,32 +170,6 @@ pub struct HummockManager { pub type HummockManagerRef = Arc; -/// Commit multiple `ValTransaction`s to state store and upon success update the local in-mem state -/// by the way -/// After called, the `ValTransaction` will be dropped. -macro_rules! commit_multi_var { - ($hummock_mgr:expr, $context_id:expr, $trx_extern_part:expr, $($val_txn:expr),*) => { - { - async { - let mut trx = $trx_extern_part; - // Apply the change in `ValTransaction` to trx - $( - $val_txn.apply_to_txn(&mut trx).await?; - )* - // Commit to state store - $hummock_mgr.commit_trx($hummock_mgr.env.meta_store(), trx, $context_id) - .await?; - // Upon successful commit, commit the change to local in-mem state - $( - $val_txn.commit(); - )* - Result::Ok(()) - }.await - } - }; -} -pub(crate) use commit_multi_var; - /// Acquire read lock of the lock with `lock_name`. /// The macro will use macro `function_name` to get the name of the function of method that calls /// the lock, and therefore, anyone to call this macro should ensured that the caller method has the @@ -440,6 +422,10 @@ impl HummockManager { Ok(instance) } + fn sql_meta_store(&self) -> Option { + self.env.sql_meta_store() + } + /// Load state from meta store. #[named] async fn load_meta_store_state(&self) -> Result<()> { @@ -458,27 +444,67 @@ impl HummockManager { compaction_guard: &mut RwLockWriteGuard<'_, Compaction>, versioning_guard: &mut RwLockWriteGuard<'_, Versioning>, ) -> Result<()> { - let compaction_statuses = CompactStatus::list(self.env.meta_store()) - .await? - .into_iter() - .map(|cg| (cg.compaction_group_id(), cg)) - .collect::>(); + use sea_orm::EntityTrait; + let sql_meta_store = self.sql_meta_store(); + let compaction_statuses: BTreeMap = match &sql_meta_store + { + None => CompactStatus::list(self.env.meta_store()) + .await? + .into_iter() + .map(|cg| (cg.compaction_group_id(), cg)) + .collect(), + Some(sql_meta_store) => compaction_status::Entity::find() + .all(&sql_meta_store.conn) + .await + .map_err(MetadataModelError::from)? + .into_iter() + .map(|m| (m.compaction_group_id as CompactionGroupId, m.into())) + .collect(), + }; if !compaction_statuses.is_empty() { compaction_guard.compaction_statuses = compaction_statuses; } - compaction_guard.compact_task_assignment = - CompactTaskAssignment::list(self.env.meta_store()) + + compaction_guard.compact_task_assignment = match &sql_meta_store { + None => CompactTaskAssignment::list(self.env.meta_store()) .await? .into_iter() .map(|assigned| (assigned.key().unwrap(), assigned)) - .collect(); - - let hummock_version_deltas: BTreeMap<_, _> = - HummockVersionDelta::list(self.env.meta_store()) - .await? + .collect(), + Some(sql_meta_store) => compaction_task::Entity::find() + .all(&sql_meta_store.conn) + .await + .map_err(MetadataModelError::from)? .into_iter() - .map(|version_delta| (version_delta.id, version_delta)) - .collect(); + .map(|m| (m.id as HummockCompactionTaskId, m.into())) + .collect(), + }; + + let hummock_version_deltas: BTreeMap = + match &sql_meta_store { + None => HummockVersionDelta::list(self.env.meta_store()) + .await? + .into_iter() + .map(|version_delta| (version_delta.id, version_delta)) + .collect(), + Some(sql_meta_store) => { + use risingwave_pb::hummock::PbHummockVersionDelta; + hummock_version_delta::Entity::find() + .all(&sql_meta_store.conn) + .await + .map_err(MetadataModelError::from)? + .into_iter() + .map(|m| { + ( + m.id as HummockVersionId, + HummockVersionDelta::from_persisted_protobuf( + &PbHummockVersionDelta::from(m), + ), + ) + }) + .collect() + } + }; let checkpoint = self.try_read_checkpoint().await?; let mut redo_state = if let Some(c) = checkpoint { @@ -492,10 +518,6 @@ impl HummockManager { .default_compaction_config(); let checkpoint_version = create_init_version(default_compaction_config); tracing::info!("init hummock version checkpoint"); - // This write to meta store is idempotent. So if `write_checkpoint` fails, restarting meta node is fine. - HummockVersionStats::default() - .insert(self.env.meta_store()) - .await?; versioning_guard.checkpoint = HummockVersionCheckpoint { version: checkpoint_version.clone(), stale_objects: Default::default(), @@ -503,17 +525,28 @@ impl HummockManager { self.write_checkpoint(&versioning_guard.checkpoint).await?; checkpoint_version }; - - versioning_guard.version_stats = HummockVersionStats::list(self.env.meta_store()) - .await? - .into_iter() - .next() - .expect("should contain exact one item"); for version_delta in hummock_version_deltas.values() { if version_delta.prev_id == redo_state.id { redo_state.apply_version_delta(version_delta); } } + versioning_guard.version_stats = match &sql_meta_store { + None => HummockVersionStats::list(self.env.meta_store()) + .await? + .into_iter() + .next(), + Some(sql_meta_store) => hummock_version_stats::Entity::find() + .one(&sql_meta_store.conn) + .await + .map_err(MetadataModelError::from)? + .map(HummockVersionStats::from), + } + .unwrap_or_else(|| HummockVersionStats { + // version_stats.hummock_version_id is always 0 in meta store. + hummock_version_id: 0, + ..Default::default() + }); + self.latest_snapshot.store( HummockSnapshot { committed_epoch: redo_state.max_committed_epoch, @@ -525,16 +558,35 @@ impl HummockManager { versioning_guard.branched_ssts = versioning_guard.current_version.build_branched_sst_info(); versioning_guard.hummock_version_deltas = hummock_version_deltas; - versioning_guard.pinned_versions = HummockPinnedVersion::list(self.env.meta_store()) - .await? - .into_iter() - .map(|p| (p.context_id, p)) - .collect(); - versioning_guard.pinned_snapshots = HummockPinnedSnapshot::list(self.env.meta_store()) - .await? - .into_iter() - .map(|p| (p.context_id, p)) - .collect(); + versioning_guard.pinned_versions = match &sql_meta_store { + None => HummockPinnedVersion::list(self.env.meta_store()) + .await? + .into_iter() + .map(|p| (p.context_id, p)) + .collect(), + Some(sql_meta_store) => hummock_pinned_version::Entity::find() + .all(&sql_meta_store.conn) + .await + .map_err(MetadataModelError::from)? + .into_iter() + .map(|m| (m.context_id as HummockContextId, m.into())) + .collect(), + }; + + versioning_guard.pinned_snapshots = match &sql_meta_store { + None => HummockPinnedSnapshot::list(self.env.meta_store()) + .await? + .into_iter() + .map(|p| (p.context_id, p)) + .collect(), + Some(sql_meta_store) => hummock_pinned_snapshot::Entity::find() + .all(&sql_meta_store.conn) + .await + .map_err(MetadataModelError::from)? + .into_iter() + .map(|m| (m.context_id as HummockContextId, m.into())) + .collect(), + }; versioning_guard.objects_to_delete.clear(); versioning_guard.mark_objects_for_deletion(); @@ -545,27 +597,15 @@ impl HummockManager { Ok(()) } - /// We use worker node id as the `context_id`. - /// If the `context_id` is provided, the transaction will abort if the `context_id` is not - /// valid, which means the worker node is not a valid member of the cluster. - /// This operation is protected by mutex of compaction, so that other thread can not - /// call `release_contexts` even if it has removed `context_id` from cluster manager. - async fn commit_trx( - &self, - meta_store: &MetaStoreRef, - trx: Transaction, - context_id: Option, - ) -> Result<()> { - if let Some(context_id) = context_id { - if context_id == META_NODE_ID { - // Using the preserved meta id is allowed. - } else if !self.check_context(context_id).await? { - // The worker is not found in cluster. - return Err(Error::InvalidContext(context_id)); - } + /// Caller should hold `versioning` lock, to sync with `HummockManager::release_contexts`. + async fn check_context_with_meta_node(&self, context_id: HummockContextId) -> Result<()> { + if context_id == META_NODE_ID { + // Using the preserved meta id is allowed. + } else if !self.check_context(context_id).await? { + // The worker is not found in cluster. + return Err(Error::InvalidContext(context_id)); } - - meta_store.txn(trx).await.map_err(Into::into) + Ok(()) } /// Pin the current greatest hummock version. The pin belongs to `context_id` @@ -573,9 +613,14 @@ impl HummockManager { #[named] pub async fn pin_version(&self, context_id: HummockContextId) -> Result { let mut versioning_guard = write_lock!(self, versioning).await; + self.check_context_with_meta_node(context_id).await?; let _timer = start_measure_real_process_timer!(self); let versioning = versioning_guard.deref_mut(); - let mut pinned_versions = BTreeMapTransaction::new(&mut versioning.pinned_versions); + let mut pinned_versions = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning.pinned_versions,) + ); let mut context_pinned_version = pinned_versions.new_entry_txn_or_default( context_id, HummockPinnedVersion { @@ -590,9 +635,8 @@ impl HummockManager { { context_pinned_version.min_pinned_id = version_id; commit_multi_var!( - self, - Some(context_id), - Transaction::default(), + self.env.meta_store(), + self.sql_meta_store(), context_pinned_version )?; trigger_pin_unpin_version_state(&self.metrics, &versioning.pinned_versions); @@ -617,9 +661,14 @@ impl HummockManager { unpin_before: HummockVersionId, ) -> Result<()> { let mut versioning_guard = write_lock!(self, versioning).await; + self.check_context_with_meta_node(context_id).await?; let _timer = start_measure_real_process_timer!(self); let versioning = versioning_guard.deref_mut(); - let mut pinned_versions = BTreeMapTransaction::new(&mut versioning.pinned_versions); + let mut pinned_versions = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning.pinned_versions,) + ); let mut context_pinned_version = pinned_versions.new_entry_txn_or_default( context_id, HummockPinnedVersion { @@ -635,9 +684,8 @@ impl HummockManager { ); context_pinned_version.min_pinned_id = unpin_before; commit_multi_var!( - self, - Some(context_id), - Transaction::default(), + self.env.meta_store(), + self.sql_meta_store(), context_pinned_version )?; trigger_pin_unpin_version_state(&self.metrics, &versioning.pinned_versions); @@ -659,7 +707,12 @@ impl HummockManager { ) -> Result { let snapshot = self.latest_snapshot.load(); let mut guard = write_lock!(self, versioning).await; - let mut pinned_snapshots = BTreeMapTransaction::new(&mut guard.pinned_snapshots); + self.check_context_with_meta_node(context_id).await?; + let mut pinned_snapshots = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut guard.pinned_snapshots,) + ); let mut context_pinned_snapshot = pinned_snapshots.new_entry_txn_or_default( context_id, HummockPinnedSnapshot { @@ -671,9 +724,8 @@ impl HummockManager { if context_pinned_snapshot.minimal_pinned_snapshot == INVALID_EPOCH { context_pinned_snapshot.minimal_pinned_snapshot = epoch_to_pin; commit_multi_var!( - self, - Some(context_id), - Transaction::default(), + self.env.meta_store(), + self.sql_meta_store(), context_pinned_snapshot )?; } @@ -685,8 +737,13 @@ impl HummockManager { pub async fn pin_snapshot(&self, context_id: HummockContextId) -> Result { let snapshot = self.latest_snapshot.load(); let mut guard = write_lock!(self, versioning).await; + self.check_context_with_meta_node(context_id).await?; let _timer = start_measure_real_process_timer!(self); - let mut pinned_snapshots = BTreeMapTransaction::new(&mut guard.pinned_snapshots); + let mut pinned_snapshots = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut guard.pinned_snapshots,) + ); let mut context_pinned_snapshot = pinned_snapshots.new_entry_txn_or_default( context_id, HummockPinnedSnapshot { @@ -697,9 +754,8 @@ impl HummockManager { if context_pinned_snapshot.minimal_pinned_snapshot == INVALID_EPOCH { context_pinned_snapshot.minimal_pinned_snapshot = snapshot.committed_epoch; commit_multi_var!( - self, - Some(context_id), - Transaction::default(), + self.env.meta_store(), + self.sql_meta_store(), context_pinned_snapshot )?; trigger_pin_unpin_snapshot_state(&self.metrics, &guard.pinned_snapshots); @@ -715,14 +771,18 @@ impl HummockManager { #[named] pub async fn unpin_snapshot(&self, context_id: HummockContextId) -> Result<()> { let mut versioning_guard = write_lock!(self, versioning).await; + self.check_context_with_meta_node(context_id).await?; let _timer = start_measure_real_process_timer!(self); - let mut pinned_snapshots = BTreeMapTransaction::new(&mut versioning_guard.pinned_snapshots); + let mut pinned_snapshots = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning_guard.pinned_snapshots,) + ); let release_snapshot = pinned_snapshots.remove(context_id); if release_snapshot.is_some() { commit_multi_var!( - self, - Some(context_id), - Transaction::default(), + self.env.meta_store(), + self.sql_meta_store(), pinned_snapshots )?; trigger_pin_unpin_snapshot_state(&self.metrics, &versioning_guard.pinned_snapshots); @@ -745,6 +805,7 @@ impl HummockManager { hummock_snapshot: HummockSnapshot, ) -> Result<()> { let mut versioning_guard = write_lock!(self, versioning).await; + self.check_context_with_meta_node(context_id).await?; let _timer = start_measure_real_process_timer!(self); // Use the max_committed_epoch in storage as the snapshot ts so only committed changes are // visible in the snapshot. @@ -757,7 +818,11 @@ impl HummockManager { } let last_read_epoch = std::cmp::min(snapshot_committed_epoch, max_committed_epoch); - let mut pinned_snapshots = BTreeMapTransaction::new(&mut versioning_guard.pinned_snapshots); + let mut pinned_snapshots = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning_guard.pinned_snapshots,) + ); let mut context_pinned_snapshot = pinned_snapshots.new_entry_txn_or_default( context_id, HummockPinnedSnapshot { @@ -773,9 +838,8 @@ impl HummockManager { { context_pinned_snapshot.minimal_pinned_snapshot = last_read_epoch; commit_multi_var!( - self, - Some(context_id), - Transaction::default(), + self.env.meta_store(), + self.sql_meta_store(), context_pinned_snapshot )?; trigger_pin_unpin_snapshot_state(&self.metrics, &versioning_guard.pinned_snapshots); @@ -819,11 +883,7 @@ impl HummockManager { let start_time = Instant::now(); // StoredIdGenerator already implements ids pre-allocation by ID_PREALLOCATE_INTERVAL. - let task_id = self - .env - .id_gen_manager() - .generate::<{ IdCategory::HummockCompactionTask }>() - .await?; + let task_id = next_compaction_task_id(&self.env).await?; // When the last table of a compaction group is deleted, the compaction group (and its // config) is destroyed as well. Then a compaction task for this group may come later and @@ -846,7 +906,11 @@ impl HummockManager { let mut compact_status = match compaction.compaction_statuses.get_mut(&compaction_group_id) { - Some(c) => VarTransaction::new(c), + Some(c) => create_trx_wrapper!( + self.sql_meta_store(), + VarTransactionWrapper, + VarTransaction::new(c) + ), None => { return Ok(None); } @@ -966,8 +1030,11 @@ impl HummockManager { compact_task.table_watermarks = current_version.safe_epoch_table_watermarks(&compact_task.existing_table_ids); - let mut compact_task_assignment = - BTreeMapTransaction::new(&mut compaction.compact_task_assignment); + let mut compact_task_assignment = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut compaction.compact_task_assignment,) + ); compact_task_assignment.insert( compact_task.task_id, CompactTaskAssignment { @@ -979,9 +1046,8 @@ impl HummockManager { // We are using a single transaction to ensure that each task has progress when it is // created. commit_multi_var!( - self, - None, - Transaction::default(), + self.env.meta_store(), + self.sql_meta_store(), compact_status, compact_task_assignment )?; @@ -1096,12 +1162,20 @@ impl HummockManager { compaction_config: &CompactionConfig, ) -> Result<()> { if !compaction_statuses.contains_key(&compaction_group_id) { - let mut compact_statuses = BTreeMapTransaction::new(compaction_statuses); + let mut compact_statuses = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(compaction_statuses) + ); let new_compact_status = compact_statuses.new_entry_insert_txn( compaction_group_id, CompactStatus::new(compaction_group_id, compaction_config.max_level), ); - commit_multi_var!(self, None, Transaction::default(), new_compact_status)?; + commit_multi_var!( + self.env.meta_store(), + self.sql_meta_store(), + new_compact_status + )?; } Ok(()) @@ -1203,10 +1277,16 @@ impl HummockManager { let compaction = compaction_guard.deref_mut(); let start_time = Instant::now(); let original_keys = compaction.compaction_statuses.keys().cloned().collect_vec(); - let mut compact_statuses = BTreeMapTransaction::new(&mut compaction.compaction_statuses); - let mut compact_task_assignment = - BTreeMapTransaction::new(&mut compaction.compact_task_assignment); - + let mut compact_statuses = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut compaction.compaction_statuses,) + ); + let mut compact_task_assignment = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut compaction.compact_task_assignment,) + ); // remove task_assignment let mut compact_task = if let Some(input_task) = trivial_move_compact_task { input_task @@ -1291,9 +1371,16 @@ impl HummockManager { false }; if is_success { - let mut hummock_version_deltas = - BTreeMapTransaction::new(&mut versioning.hummock_version_deltas); - let mut branched_ssts = BTreeMapTransaction::new(&mut versioning.branched_ssts); + let mut hummock_version_deltas = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning.hummock_version_deltas,) + ); + let mut branched_ssts = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning.branched_ssts) + ); let version_delta = gen_version_delta( &mut hummock_version_deltas, &mut branched_ssts, @@ -1301,7 +1388,11 @@ impl HummockManager { &compact_task, deterministic_mode, ); - let mut version_stats = VarTransaction::new(&mut versioning.version_stats); + let mut version_stats = create_trx_wrapper!( + self.sql_meta_store(), + VarTransactionWrapper, + VarTransaction::new(&mut versioning.version_stats,) + ); if let Some(table_stats_change) = &table_stats_change { add_prost_table_stats_map(&mut version_stats.table_stats, table_stats_change); } @@ -1310,9 +1401,8 @@ impl HummockManager { // recover to a correct state after restarting meta-node. current_version.apply_version_delta(&version_delta); commit_multi_var!( - self, - None, - Transaction::default(), + self.env.meta_store(), + self.sql_meta_store(), compact_statuses, compact_task_assignment, hummock_version_deltas, @@ -1331,9 +1421,8 @@ impl HummockManager { } else { // The compaction task is cancelled or failed. commit_multi_var!( - self, - None, - Transaction::default(), + self.env.meta_store(), + self.sql_meta_store(), compact_statuses, compact_task_assignment )?; @@ -1433,10 +1522,14 @@ impl HummockManager { } let old_version = &versioning.current_version; - let mut new_version_delta = BTreeMapEntryTransaction::new_insert( - &mut versioning.hummock_version_deltas, - old_version.id + 1, - build_version_delta_after_version(old_version), + let mut new_version_delta = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapEntryTransactionWrapper, + BTreeMapEntryTransaction::new_insert( + &mut versioning.hummock_version_deltas, + old_version.id + 1, + build_version_delta_after_version(old_version), + ) ); new_version_delta.max_committed_epoch = epoch; new_version_delta.new_table_watermarks = new_table_watermarks; @@ -1494,12 +1587,12 @@ impl HummockManager { } } } - let mut new_sst_id = self - .env - .id_gen_manager() - .generate_interval::<{ IdCategory::HummockSstableId }>(new_sst_id_number as u64) - .await?; - let mut branched_ssts = BTreeMapTransaction::<'_, _, _>::new(&mut versioning.branched_ssts); + let mut new_sst_id = next_sstable_object_id(&self.env, new_sst_id_number).await?; + let mut branched_ssts = create_trx_wrapper!( + self.sql_meta_store(), + BTreeMapTransactionWrapper, + BTreeMapTransaction::new(&mut versioning.branched_ssts) + ); let original_sstables = std::mem::take(&mut sstables); sstables.reserve_exact(original_sstables.len() - incorrect_ssts.len() + new_sst_id_number); let mut incorrect_ssts = incorrect_ssts.into_iter(); @@ -1572,7 +1665,11 @@ impl HummockManager { new_hummock_version.apply_version_delta(new_version_delta.deref()); // Apply stats changes. - let mut version_stats = VarTransaction::new(&mut versioning.version_stats); + let mut version_stats = create_trx_wrapper!( + self.sql_meta_store(), + VarTransactionWrapper, + VarTransaction::new(&mut versioning.version_stats) + ); add_prost_table_stats_map(&mut version_stats.table_stats, &table_stats_change); purge_prost_table_stats(&mut version_stats.table_stats, &new_hummock_version); for (table_id, stats) in &table_stats_change { @@ -1584,11 +1681,9 @@ impl HummockManager { .with_label_values(&[table_id_str.as_str()]) .inc_by(stats_value as u64); } - commit_multi_var!( - self, - None, - Transaction::default(), + self.env.meta_store(), + self.sql_meta_store(), new_version_delta, version_stats )?; @@ -1675,11 +1770,7 @@ impl HummockManager { } pub async fn get_new_sst_ids(&self, number: u32) -> Result { - let start_id = self - .env - .id_gen_manager() - .generate_interval::<{ IdCategory::HummockSstableId }>(number as u64) - .await?; + let start_id = next_sstable_object_id(&self.env, number).await?; Ok(SstObjectIdRange::new(start_id, start_id + number as u64)) } @@ -1688,7 +1779,6 @@ impl HummockManager { read_lock!(self, versioning).await.min_pinned_version_id() } - // TODO: use proc macro to call check_state_consistency #[named] #[cfg(test)] pub async fn check_state_consistency(&self) { @@ -3057,7 +3147,7 @@ pub enum TableAlignRule { } fn drop_sst( - branched_ssts: &mut BTreeMapTransaction<'_, HummockSstableObjectId, BranchedSstInfo>, + branched_ssts: &mut BTreeMapTransactionWrapper<'_, HummockSstableObjectId, BranchedSstInfo>, group_id: CompactionGroupId, object_id: HummockSstableObjectId, sst_id: HummockSstableId, @@ -3080,8 +3170,8 @@ fn drop_sst( } fn gen_version_delta<'a>( - txn: &mut BTreeMapTransaction<'a, HummockVersionId, HummockVersionDelta>, - branched_ssts: &mut BTreeMapTransaction<'a, HummockSstableObjectId, BranchedSstInfo>, + txn: &mut BTreeMapTransactionWrapper<'a, HummockVersionId, HummockVersionDelta>, + branched_ssts: &mut BTreeMapTransactionWrapper<'a, HummockSstableObjectId, BranchedSstInfo>, old_version: &HummockVersion, compact_task: &CompactTask, deterministic_mode: bool, @@ -3231,7 +3321,9 @@ use tokio::sync::mpsc::error::SendError; use super::compaction::selector::EmergencySelector; use super::compaction::CompactionSelector; +use crate::controller::SqlMetaStore; use crate::hummock::manager::checkpoint::HummockVersionCheckpoint; +use crate::hummock::sequence::next_sstable_object_id; #[derive(Debug, Default)] pub struct CompactionState { diff --git a/src/meta/src/hummock/manager/sequence.rs b/src/meta/src/hummock/manager/sequence.rs index 50ee756de7f0f..c1cdda952c6ed 100644 --- a/src/meta/src/hummock/manager/sequence.rs +++ b/src/meta/src/hummock/manager/sequence.rs @@ -12,10 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![allow(dead_code)] - use std::collections::HashMap; -use std::num::NonZeroU32; +use std::fmt::Display; use std::sync::LazyLock; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; @@ -23,7 +21,8 @@ use risingwave_meta_model_v2::hummock_sequence; use sea_orm::{ActiveModelTrait, ActiveValue, DatabaseConnection, EntityTrait, TransactionTrait}; use tokio::sync::Mutex; -use crate::MetaResult; +use crate::hummock::error::Result; +use crate::manager::{IdCategory, MetaSrvEnv}; const COMPACTION_TASK_ID: &str = "compaction_task"; const COMPACTION_GROUP_ID: &str = "compaction_group"; @@ -51,45 +50,101 @@ impl SequenceGenerator { /// Returns start, indicates range [start, start + num). /// /// Despite being a serial function, its infrequent invocation allows for acceptable performance. - pub async fn next_interval(&self, ident: &str, num: NonZeroU32) -> MetaResult { + /// + /// If num is 0, the next seq is returned just like num is 1, but caller must not use this seq. + pub async fn next_interval(&self, ident: &str, num: u32) -> Result { // TODO: add pre-allocation if necessary let guard = self.db.lock().await; let txn = guard.begin().await?; let model: Option = hummock_sequence::Entity::find_by_id(ident.to_string()) .one(&txn) - .await - .unwrap(); + .await?; let start_seq = match model { None => { - let init = SEQ_INIT + let init: u64 = SEQ_INIT .get(ident) .copied() - .unwrap_or_else(|| panic!("seq {ident} not found")); + .unwrap_or_else(|| panic!("seq {ident} not found")) + as u64; let active_model = hummock_sequence::ActiveModel { name: ActiveValue::set(ident.into()), - seq: ActiveValue::set(init + num.get() as i64), + seq: ActiveValue::set(init.checked_add(num as _).unwrap().try_into().unwrap()), }; active_model.insert(&txn).await?; init } Some(model) => { - let start_seq = model.seq; - let mut active_model: hummock_sequence::ActiveModel = model.into(); - active_model.seq = ActiveValue::set(start_seq + num.get() as i64); - active_model.update(&txn).await?; + let start_seq: u64 = model.seq as u64; + if num > 0 { + let mut active_model: hummock_sequence::ActiveModel = model.into(); + active_model.seq = ActiveValue::set( + start_seq.checked_add(num as _).unwrap().try_into().unwrap(), + ); + active_model.update(&txn).await?; + } start_seq } }; - txn.commit().await?; - Ok(u64::try_from(start_seq).unwrap_or_else(|_| panic!("seq {ident} overflow"))) + if num > 0 { + txn.commit().await?; + } + Ok(start_seq) + } +} + +pub async fn next_compaction_task_id(env: &MetaSrvEnv) -> Result { + match env.hummock_seq.clone() { + None => env + .id_gen_manager() + .generate::<{ IdCategory::HummockCompactionTask }>() + .await + .map_err(Into::into), + Some(seq) => seq.next_interval(COMPACTION_TASK_ID, 1).await, + } +} + +pub async fn next_meta_backup_id(env: &MetaSrvEnv) -> Result { + match env.hummock_seq.clone() { + None => env + .id_gen_manager() + .generate::<{ IdCategory::Backup }>() + .await + .map_err(Into::into), + Some(seq) => seq.next_interval(META_BACKUP_ID, 1).await, + } +} + +pub async fn next_compaction_group_id(env: &MetaSrvEnv) -> Result { + match env.hummock_seq.clone() { + None => env + .id_gen_manager() + .generate::<{ IdCategory::CompactionGroup }>() + .await + .map_err(Into::into), + Some(seq) => seq.next_interval(COMPACTION_GROUP_ID, 1).await, + } +} + +pub async fn next_sstable_object_id( + env: &MetaSrvEnv, + num: impl TryInto + Display + Copy, +) -> Result { + let num: u32 = num + .try_into() + .unwrap_or_else(|_| panic!("fail to convert {num} into u32")); + match env.hummock_seq.clone() { + None => env + .id_gen_manager() + .generate_interval::<{ IdCategory::HummockSstableId }>(num as u64) + .await + .map_err(Into::into), + Some(seq) => seq.next_interval(SSTABLE_OBJECT_ID, num).await, } } #[cfg(test)] mod tests { - use std::num::NonZeroU32; - use crate::controller::SqlMetaStore; use crate::hummock::manager::sequence::{SequenceGenerator, COMPACTION_TASK_ID}; @@ -99,23 +154,8 @@ mod tests { let store = SqlMetaStore::for_test().await; let conn = store.conn.clone(); let s = SequenceGenerator::new(conn); - assert_eq!( - 1, - s.next_interval(COMPACTION_TASK_ID, NonZeroU32::new(1).unwrap()) - .await - .unwrap() - ); - assert_eq!( - 2, - s.next_interval(COMPACTION_TASK_ID, NonZeroU32::new(10).unwrap()) - .await - .unwrap() - ); - assert_eq!( - 12, - s.next_interval(COMPACTION_TASK_ID, NonZeroU32::new(10).unwrap()) - .await - .unwrap() - ); + assert_eq!(1, s.next_interval(COMPACTION_TASK_ID, 1).await.unwrap()); + assert_eq!(2, s.next_interval(COMPACTION_TASK_ID, 10).await.unwrap()); + assert_eq!(12, s.next_interval(COMPACTION_TASK_ID, 10).await.unwrap()); } } diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 804d7381c858e..ded4c75124405 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -47,7 +47,7 @@ use crate::hummock::compaction::CompactStatus; use crate::hummock::error::Error; use crate::hummock::test_utils::*; use crate::hummock::{CommitEpochInfo, HummockManager, HummockManagerRef}; -use crate::manager::WorkerId; +use crate::manager::{MetaSrvEnv, WorkerId}; use crate::model::MetadataModel; use crate::rpc::metrics::MetaMetrics; @@ -99,6 +99,40 @@ fn get_compaction_group_object_ids( .collect_vec() } +async fn list_pinned_snapshot_from_meta_store(env: &MetaSrvEnv) -> Vec { + match env.sql_meta_store() { + None => HummockPinnedSnapshot::list(env.meta_store()).await.unwrap(), + Some(sql_meta_store) => { + use risingwave_meta_model_v2::hummock_pinned_snapshot; + use sea_orm::EntityTrait; + hummock_pinned_snapshot::Entity::find() + .all(&sql_meta_store.conn) + .await + .unwrap() + .into_iter() + .map(Into::into) + .collect() + } + } +} + +async fn list_pinned_version_from_meta_store(env: &MetaSrvEnv) -> Vec { + match env.sql_meta_store() { + None => HummockPinnedVersion::list(env.meta_store()).await.unwrap(), + Some(sql_meta_store) => { + use risingwave_meta_model_v2::hummock_pinned_version; + use sea_orm::EntityTrait; + hummock_pinned_version::Entity::find() + .all(&sql_meta_store.conn) + .await + .unwrap() + .into_iter() + .map(Into::into) + .collect() + } + } +} + #[tokio::test] async fn test_unpin_snapshot_before() { let (env, hummock_manager, _cluster_manager, worker_node) = setup_compute_env(80).await; @@ -108,7 +142,7 @@ async fn test_unpin_snapshot_before() { for _ in 0..2 { let pin_result = hummock_manager.pin_snapshot(context_id).await.unwrap(); assert_eq!(pin_result.committed_epoch, epoch); - let pinned_snapshots = HummockPinnedSnapshot::list(env.meta_store()).await.unwrap(); + let pinned_snapshots = list_pinned_snapshot_from_meta_store(&env).await; assert_eq!(pinned_snapshots[0].context_id, context_id); assert_eq!( pinned_snapshots[0].minimal_pinned_snapshot, @@ -129,7 +163,7 @@ async fn test_unpin_snapshot_before() { .await .unwrap(); assert_eq!( - pin_snapshots_epoch(&HummockPinnedSnapshot::list(env.meta_store()).await.unwrap()), + pin_snapshots_epoch(&list_pinned_snapshot_from_meta_store(&env).await), vec![epoch] ); } @@ -147,7 +181,7 @@ async fn test_unpin_snapshot_before() { .await .unwrap(); assert_eq!( - pin_snapshots_epoch(&HummockPinnedSnapshot::list(env.meta_store()).await.unwrap()), + pin_snapshots_epoch(&list_pinned_snapshot_from_meta_store(&env).await), vec![epoch] ); } @@ -389,7 +423,7 @@ async fn test_release_context_resource() { let context_id_2 = worker_node_2.id; assert_eq!( - pin_versions_sum(&HummockPinnedVersion::list(env.meta_store()).await.unwrap()), + pin_versions_sum(&list_pinned_version_from_meta_store(&env).await), 0 ); hummock_manager.pin_version(context_id_1).await.unwrap(); @@ -397,24 +431,18 @@ async fn test_release_context_resource() { hummock_manager.pin_snapshot(context_id_1).await.unwrap(); hummock_manager.pin_snapshot(context_id_2).await.unwrap(); assert_eq!( - pin_versions_sum(&HummockPinnedVersion::list(env.meta_store()).await.unwrap()), - 2 - ); - assert_eq!( - HummockPinnedSnapshot::list(env.meta_store()) - .await - .unwrap() - .len(), + pin_versions_sum(&list_pinned_version_from_meta_store(&env).await), 2 ); + assert_eq!(list_pinned_version_from_meta_store(&env).await.len(), 2); hummock_manager .release_contexts(&vec![context_id_1]) .await .unwrap(); - let pinned_versions = HummockPinnedVersion::list(env.meta_store()).await.unwrap(); + let pinned_versions = list_pinned_version_from_meta_store(&env).await; assert_eq!(pin_versions_sum(&pinned_versions), 1); assert_eq!(pinned_versions[0].context_id, context_id_2); - let pinned_snapshots = HummockPinnedSnapshot::list(env.meta_store()).await.unwrap(); + let pinned_snapshots = list_pinned_snapshot_from_meta_store(&env).await; assert_eq!(pinned_snapshots[0].context_id, context_id_2); // it's OK to call again hummock_manager @@ -426,7 +454,7 @@ async fn test_release_context_resource() { .await .unwrap(); assert_eq!( - pin_versions_sum(&HummockPinnedVersion::list(env.meta_store()).await.unwrap()), + pin_versions_sum(&list_pinned_version_from_meta_store(&env).await), 0 ); } diff --git a/src/meta/src/hummock/manager/utils.rs b/src/meta/src/hummock/manager/utils.rs new file mode 100644 index 0000000000000..9ddc9699cb5e3 --- /dev/null +++ b/src/meta/src/hummock/manager/utils.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. + +/// Commit multiple `ValTransaction`s to state store and upon success update the local in-mem state +/// by the way +/// After called, the `ValTransaction` will be dropped. +macro_rules! commit_multi_var { + ($meta_store:expr, $sql_meta_store:expr, $($val_txn:expr),*) => { + { + async { + match &$sql_meta_store { + None => { + use crate::storage::Transaction; + let mut trx = Transaction::default(); + $( + $val_txn.as_v1_ref().apply_to_txn(&mut trx).await?; + )* + $meta_store.txn(trx).await?; + $( + $val_txn.into_v1().commit(); + )* + Result::Ok(()) + } + Some(sql_meta_store) => { + use sea_orm::TransactionTrait; + use crate::model::MetadataModelError; + let mut trx = sql_meta_store.conn.begin().await.map_err(MetadataModelError::from)?; + $( + $val_txn.as_v2_ref().apply_to_txn(&mut trx).await?; + )* + trx.commit().await.map_err(MetadataModelError::from)?; + $( + $val_txn.into_v2().commit(); + )* + Result::Ok(()) + } + } + }.await + } + }; +} +pub(crate) use commit_multi_var; + +macro_rules! create_trx_wrapper { + ($sql_meta_store:expr, $wrapper:ident, $inner:expr) => {{ + match &$sql_meta_store { + None => $wrapper::V1($inner), + Some(_) => $wrapper::V2($inner), + } + }}; +} + +pub(crate) use create_trx_wrapper; diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index b06661ec0e6cb..bca0d13cfd241 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -38,12 +38,12 @@ use risingwave_pb::meta::subscribe_response::{Info, Operation}; use crate::hummock::error::Result; use crate::hummock::manager::checkpoint::HummockVersionCheckpoint; use crate::hummock::manager::worker::{HummockManagerEvent, HummockManagerEventSender}; -use crate::hummock::manager::{commit_multi_var, read_lock, write_lock}; +use crate::hummock::manager::{commit_multi_var, create_trx_wrapper, read_lock, write_lock}; use crate::hummock::metrics_utils::{trigger_safepoint_stat, trigger_write_stop_stats}; use crate::hummock::model::CompactionGroup; use crate::hummock::HummockManager; -use crate::model::{ValTransaction, VarTransaction}; -use crate::storage::Transaction; +use crate::model::{VarTransaction, VarTransactionWrapper}; +use crate::storage::MetaStore; use crate::MetaResult; /// `HummockVersionSafePoint` prevents hummock versions GE than it from being GC. @@ -290,11 +290,17 @@ impl HummockManager { #[named] pub async fn rebuild_table_stats(&self) -> Result<()> { + use crate::model::ValTransaction; let mut versioning = write_lock!(self, versioning).await; let new_stats = rebuild_table_stats(&versioning.current_version); - let mut version_stats = VarTransaction::new(&mut versioning.version_stats); - *version_stats = new_stats; - commit_multi_var!(self, None, Transaction::default(), version_stats)?; + let mut version_stats = create_trx_wrapper!( + self.sql_meta_store(), + VarTransactionWrapper, + VarTransaction::new(&mut versioning.version_stats) + ); + // version_stats.hummock_version_id is always 0 in meta store. + version_stats.table_stats = new_stats.table_stats; + commit_multi_var!(self.env.meta_store(), self.sql_meta_store(), version_stats)?; Ok(()) } } diff --git a/src/meta/src/hummock/model/ext/hummock.rs b/src/meta/src/hummock/model/ext/hummock.rs index d64c25b4f2fa1..b7d136776a952 100644 --- a/src/meta/src/hummock/model/ext/hummock.rs +++ b/src/meta/src/hummock/model/ext/hummock.rs @@ -18,12 +18,15 @@ use risingwave_meta_model_v2::compaction_config::CompactionConfig; use risingwave_meta_model_v2::compaction_status::LevelHandlers; use risingwave_meta_model_v2::compaction_task::CompactionTask; use risingwave_meta_model_v2::hummock_version_delta::FullVersionDelta; +use risingwave_meta_model_v2::hummock_version_stats::TableStats; use risingwave_meta_model_v2::{ compaction_config, compaction_status, compaction_task, hummock_pinned_snapshot, - hummock_pinned_version, hummock_version_delta, CompactionGroupId, CompactionTaskId, - HummockVersionId, WorkerId, + hummock_pinned_version, hummock_version_delta, hummock_version_stats, CompactionGroupId, + CompactionTaskId, HummockVersionId, WorkerId, +}; +use risingwave_pb::hummock::{ + CompactTaskAssignment, HummockPinnedSnapshot, HummockPinnedVersion, HummockVersionStats, }; -use risingwave_pb::hummock::{CompactTaskAssignment, HummockPinnedSnapshot, HummockPinnedVersion}; use sea_orm::sea_query::OnConflict; use sea_orm::ActiveValue::Set; use sea_orm::EntityTrait; @@ -182,6 +185,32 @@ impl Transactional for HummockPinnedSnapshot { } } +#[async_trait::async_trait] +impl Transactional for HummockVersionStats { + async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + let m = hummock_version_stats::ActiveModel { + id: Set(self.hummock_version_id as _), + stats: Set(TableStats(self.table_stats.clone())), + }; + hummock_version_stats::Entity::insert(m) + .on_conflict( + OnConflict::column(hummock_version_stats::Column::Id) + .update_columns([hummock_version_stats::Column::Stats]) + .to_owned(), + ) + .exec(trx) + .await?; + Ok(()) + } + + async fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + hummock_version_stats::Entity::delete_by_id(self.hummock_version_id as i64) + .exec(trx) + .await?; + Ok(()) + } +} + #[async_trait::async_trait] impl Transactional for HummockVersionDelta { async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 769cbbe198aff..cfdaf7a2d979f 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -25,6 +25,7 @@ use sea_orm::EntityTrait; use super::{SystemParamsManager, SystemParamsManagerRef}; use crate::controller::system_param::{SystemParamsController, SystemParamsControllerRef}; use crate::controller::SqlMetaStore; +use crate::hummock::sequence::SequenceGenerator; use crate::manager::event_log::{start_event_log_manager, EventLogMangerRef}; use crate::manager::{ IdGeneratorManager, IdGeneratorManagerRef, IdleManager, IdleManagerRef, NotificationManager, @@ -75,6 +76,8 @@ pub struct MetaSrvEnv { /// Client to connector node. `None` if endpoint unspecified or unable to connect. connector_client: Option, + pub hummock_seq: Option>, + /// options read by all services pub opts: Arc, } @@ -308,6 +311,9 @@ impl MetaSrvEnv { opts.event_log_enabled, opts.event_log_channel_max_size, )); + let hummock_seq = meta_store_sql + .clone() + .map(|m| Arc::new(SequenceGenerator::new(m.conn))); Ok(Self { id_gen_manager, @@ -323,6 +329,7 @@ impl MetaSrvEnv { cluster_first_launch, connector_client, opts: opts.into(), + hummock_seq, }) } @@ -457,6 +464,9 @@ impl MetaSrvEnv { }; let event_log_manager = Arc::new(EventLogManger::for_test()); + let hummock_seq = meta_store_sql + .clone() + .map(|m| Arc::new(SequenceGenerator::new(m.conn))); Self { id_gen_manager, @@ -472,6 +482,7 @@ impl MetaSrvEnv { cluster_first_launch, connector_client: None, opts, + hummock_seq, } } } diff --git a/src/meta/src/model/mod.rs b/src/meta/src/model/mod.rs index 729c140888639..53b78c671c660 100644 --- a/src/meta/src/model/mod.rs +++ b/src/meta/src/model/mod.rs @@ -36,6 +36,7 @@ pub use notification::*; use prost::Message; pub use stream::*; +use crate::hummock::model::ext::Transaction as TransactionV2; use crate::storage::{MetaStore, MetaStoreError, Snapshot, Transaction}; /// A global, unique identifier of an actor @@ -670,6 +671,264 @@ impl<'a, K: Ord, V: PartialEq + Transactional, TXN> ValTransaction } } +pub enum BTreeMapTransactionWrapper<'a, K: Ord, V> { + V1(BTreeMapTransaction<'a, K, V, Transaction>), + V2(BTreeMapTransaction<'a, K, V, TransactionV2>), +} + +impl<'a, K: Ord + Debug, V: Clone> BTreeMapTransactionWrapper<'a, K, V> { + pub fn tree_ref(&self) -> &BTreeMap { + match self { + BTreeMapTransactionWrapper::V1(v) => v.tree_ref, + BTreeMapTransactionWrapper::V2(v) => v.tree_ref, + } + } + + pub fn tree_mut(&mut self) -> &mut BTreeMap { + match self { + BTreeMapTransactionWrapper::V1(v) => v.tree_ref, + BTreeMapTransactionWrapper::V2(v) => v.tree_ref, + } + } + + /// Get the value of the provided key by merging the staging value and the original value + pub fn get(&self, key: &K) -> Option<&V> { + match self { + BTreeMapTransactionWrapper::V1(v) => v.get(key), + BTreeMapTransactionWrapper::V2(v) => v.get(key), + } + } + + pub fn contains_key(&self, key: &K) -> bool { + match self { + BTreeMapTransactionWrapper::V1(v) => v.contains_key(key), + BTreeMapTransactionWrapper::V2(v) => v.contains_key(key), + } + } + + pub fn get_mut(&mut self, key: K) -> Option> { + match self { + BTreeMapTransactionWrapper::V1(v) => v.get_mut(key), + BTreeMapTransactionWrapper::V2(v) => v.get_mut(key), + } + } + + pub fn insert(&mut self, key: K, value: V) { + match self { + BTreeMapTransactionWrapper::V1(v) => v.insert(key, value), + BTreeMapTransactionWrapper::V2(v) => v.insert(key, value), + } + } + + pub fn remove(&mut self, key: K) -> Option { + match self { + BTreeMapTransactionWrapper::V1(v) => v.remove(key), + BTreeMapTransactionWrapper::V2(v) => v.remove(key), + } + } + + pub fn commit_memory(self) { + match self { + BTreeMapTransactionWrapper::V1(v) => v.commit_memory(), + BTreeMapTransactionWrapper::V2(v) => v.commit_memory(), + } + } + + pub fn new_entry_txn_or_default( + &mut self, + key: K, + default_val: V, + ) -> BTreeMapEntryTransactionWrapper<'_, K, V> { + match self { + BTreeMapTransactionWrapper::V1(v) => BTreeMapEntryTransactionWrapper::V1( + BTreeMapEntryTransaction::new(v.tree_ref, key, Some(default_val)) + .expect("default value is provided and should return `Some`"), + ), + BTreeMapTransactionWrapper::V2(v) => BTreeMapEntryTransactionWrapper::V2( + BTreeMapEntryTransaction::new(v.tree_ref, key, Some(default_val)) + .expect("default value is provided and should return `Some`"), + ), + } + } + + pub fn new_entry_insert_txn( + &mut self, + key: K, + val: V, + ) -> BTreeMapEntryTransactionWrapper<'_, K, V> { + match self { + BTreeMapTransactionWrapper::V1(v) => BTreeMapEntryTransactionWrapper::V1( + BTreeMapEntryTransaction::new_insert(v.tree_ref, key, val), + ), + BTreeMapTransactionWrapper::V2(v) => BTreeMapEntryTransactionWrapper::V2( + BTreeMapEntryTransaction::new_insert(v.tree_ref, key, val), + ), + } + } +} + +impl<'a, K: Ord + Debug, V: Clone> BTreeMapTransactionWrapper<'a, K, V> { + pub fn into_v1(self) -> BTreeMapTransaction<'a, K, V, Transaction> { + match self { + BTreeMapTransactionWrapper::V1(v) => v, + BTreeMapTransactionWrapper::V2(_) => panic!("expect V1, found V2"), + } + } + + pub fn as_v1_ref(&self) -> &BTreeMapTransaction<'a, K, V, Transaction> { + match self { + BTreeMapTransactionWrapper::V1(v) => v, + BTreeMapTransactionWrapper::V2(_) => panic!("expect V1, found V2"), + } + } + + pub fn into_v2(self) -> BTreeMapTransaction<'a, K, V, TransactionV2> { + match self { + BTreeMapTransactionWrapper::V1(_) => panic!("expect V2, found V1"), + BTreeMapTransactionWrapper::V2(v) => v, + } + } + + pub fn as_v2_ref(&self) -> &BTreeMapTransaction<'a, K, V, TransactionV2> { + match self { + BTreeMapTransactionWrapper::V1(_) => panic!("expect V2, found V1"), + BTreeMapTransactionWrapper::V2(v) => v, + } + } +} + +pub enum BTreeMapEntryTransactionWrapper<'a, K, V> { + V1(BTreeMapEntryTransaction<'a, K, V, Transaction>), + V2(BTreeMapEntryTransaction<'a, K, V, TransactionV2>), +} + +impl<'a, K: Ord + Debug, V: Clone> Deref for BTreeMapEntryTransactionWrapper<'a, K, V> { + type Target = V; + + fn deref(&self) -> &Self::Target { + match self { + BTreeMapEntryTransactionWrapper::V1(v) => v.deref(), + BTreeMapEntryTransactionWrapper::V2(v) => v.deref(), + } + } +} + +impl<'a, K: Ord + Debug, V: Clone> DerefMut for BTreeMapEntryTransactionWrapper<'a, K, V> { + fn deref_mut(&mut self) -> &mut Self::Target { + match self { + BTreeMapEntryTransactionWrapper::V1(v) => v.deref_mut(), + BTreeMapEntryTransactionWrapper::V2(v) => v.deref_mut(), + } + } +} + +impl<'a, K: Ord + Debug, V: Clone> BTreeMapEntryTransactionWrapper<'a, K, V> { + pub fn as_v1_ref(&self) -> &BTreeMapEntryTransaction<'a, K, V, Transaction> { + match self { + BTreeMapEntryTransactionWrapper::V1(v) => v, + BTreeMapEntryTransactionWrapper::V2(_) => { + panic!("expect V1, found V2") + } + } + } + + pub fn into_v1(self) -> BTreeMapEntryTransaction<'a, K, V, Transaction> { + match self { + BTreeMapEntryTransactionWrapper::V1(v) => v, + BTreeMapEntryTransactionWrapper::V2(_) => { + panic!("expect V1, found V2") + } + } + } + + pub fn as_v2_ref(&self) -> &BTreeMapEntryTransaction<'a, K, V, TransactionV2> { + match self { + BTreeMapEntryTransactionWrapper::V1(_) => { + panic!("expect V2, found V1") + } + BTreeMapEntryTransactionWrapper::V2(v) => v, + } + } + + pub fn into_v2(self) -> BTreeMapEntryTransaction<'a, K, V, TransactionV2> { + match self { + BTreeMapEntryTransactionWrapper::V1(_) => { + panic!("expect V2, found V1") + } + BTreeMapEntryTransactionWrapper::V2(v) => v, + } + } +} + +pub enum VarTransactionWrapper<'a, T: Transactional + Transactional> { + V1(VarTransaction<'a, Transaction, T>), + V2(VarTransaction<'a, TransactionV2, T>), +} + +impl<'a, T: Transactional + Transactional> + VarTransactionWrapper<'a, T> +{ + pub fn as_v1_ref(&self) -> &VarTransaction<'a, Transaction, T> { + match self { + VarTransactionWrapper::V1(v) => v, + VarTransactionWrapper::V2(_) => { + panic!("expect V1, found V2") + } + } + } + + pub fn into_v1(self) -> VarTransaction<'a, Transaction, T> { + match self { + VarTransactionWrapper::V1(v) => v, + VarTransactionWrapper::V2(_) => { + panic!("expect V1, found V2") + } + } + } + + pub fn as_v2_ref(&self) -> &VarTransaction<'a, TransactionV2, T> { + match self { + VarTransactionWrapper::V1(_) => { + panic!("expect V2, found V1") + } + VarTransactionWrapper::V2(v) => v, + } + } + + pub fn into_v2(self) -> VarTransaction<'a, TransactionV2, T> { + match self { + VarTransactionWrapper::V1(_) => { + panic!("expect V2, found V1") + } + VarTransactionWrapper::V2(v) => v, + } + } +} + +impl<'a, T: Transactional + Transactional> Deref + for VarTransactionWrapper<'a, T> +{ + type Target = T; + + fn deref(&self) -> &Self::Target { + match self { + VarTransactionWrapper::V1(v) => v.deref(), + VarTransactionWrapper::V2(v) => v.deref(), + } + } +} + +impl<'a, T: Transactional + Transactional + Clone> DerefMut + for VarTransactionWrapper<'a, T> +{ + fn deref_mut(&mut self) -> &mut Self::Target { + match self { + VarTransactionWrapper::V1(v) => v.deref_mut(), + VarTransactionWrapper::V2(v) => v.deref_mut(), + } + } +} + #[cfg(test)] mod tests { use super::*; From e6a6558e5730ec37cb6a3d99df839bc650705f63 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Wed, 10 Jan 2024 10:29:58 +0800 Subject: [PATCH 05/17] chore: add `test_decode_row` for memcomparable encoding (#14460) --- src/common/src/util/memcmp_encoding.rs | 40 ++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/src/common/src/util/memcmp_encoding.rs b/src/common/src/util/memcmp_encoding.rs index fe08686de14c7..ffcfe5d42a18b 100644 --- a/src/common/src/util/memcmp_encoding.rs +++ b/src/common/src/util/memcmp_encoding.rs @@ -623,6 +623,46 @@ mod tests { assert!(encoded_row1 < encoded_row2); } + // See also `row_value_encode_decode()` in `src/common/src/row/owned_row.rs` + #[test] + fn test_decode_row() { + let encoded: Vec = vec![ + 0, 128, 0, 0, 42, 255, 127, 255, 255, 255, 255, 255, 255, 213, 1, 0, 193, 186, 163, + 215, 255, 254, 153, 144, 144, 144, 144, 144, 255, 255, 249, 0, 1, 98, 97, 97, 97, 97, + 114, 0, 0, 6, + ]; + + let order_types = vec![ + OrderType::ascending(), + OrderType::descending(), + OrderType::ascending(), + OrderType::ascending(), + OrderType::descending(), + OrderType::ascending(), + ]; + let data_types = vec![ + DataType::Int32, + DataType::Int64, + DataType::Timestamp, + DataType::Float32, + DataType::Varchar, + DataType::Bytea, + ]; + + let result = decode_row(&encoded, &data_types, &order_types).unwrap(); + // println!("{:?}", &result); + + let expected = OwnedRow::new(vec![ + Some(ScalarImpl::Int32(42)), + Some(ScalarImpl::Int64(42)), + None, + Some(ScalarImpl::Float32(23.33.into())), + Some(ScalarImpl::Utf8("fooooo".into())), + Some(ScalarImpl::Bytea("baaaar".as_bytes().into())), + ]); + assert_eq!(&result, &expected); + } + #[test] fn test_encode_chunk() { let v10 = Some(ScalarImpl::Int32(42)); From df3eb218f32bd710bb1c5b7526ecd023d7af5f63 Mon Sep 17 00:00:00 2001 From: August Date: Wed, 10 Jan 2024 10:40:42 +0800 Subject: [PATCH 06/17] fix: missing apply of source split discovery info after replacing table (#14449) --- src/meta/src/barrier/command.rs | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 39bc3ced0023a..ac1ffb392cef8 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -994,6 +994,21 @@ impl CommandContext { init_split_assignment.clone(), ) .await?; + + // Apply the split changes in source manager. + self.barrier_manager_context + .source_manager + .drop_source_fragments(std::slice::from_ref(old_table_fragments)) + .await; + let source_fragments = new_table_fragments.stream_source_fragments(); + self.barrier_manager_context + .source_manager + .apply_source_change( + Some(source_fragments), + Some(init_split_assignment.clone()), + None, + ) + .await; } } From bfa1fbbd45a5d17589251aba56aa69b5ae62cbae Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Wed, 10 Jan 2024 10:57:30 +0800 Subject: [PATCH 07/17] fix(planner): preserve order in array-construction subquery (#14451) --- .../batch/subquery/array_construct.slt.part | 31 +++++++++++++ src/frontend/planner_test/src/lib.rs | 3 +- src/frontend/src/optimizer/mod.rs | 42 +++++++++++++++--- src/frontend/src/planner/insert.rs | 2 +- src/frontend/src/planner/relation.rs | 2 +- src/frontend/src/planner/select.rs | 43 ++++--------------- src/frontend/src/planner/set_expr.rs | 2 +- 7 files changed, 79 insertions(+), 46 deletions(-) create mode 100644 e2e_test/batch/subquery/array_construct.slt.part diff --git a/e2e_test/batch/subquery/array_construct.slt.part b/e2e_test/batch/subquery/array_construct.slt.part new file mode 100644 index 0000000000000..c51bc0337e758 --- /dev/null +++ b/e2e_test/batch/subquery/array_construct.slt.part @@ -0,0 +1,31 @@ +query T +select array(select unnest || ordinality from unnest(array['c', 'a', 'b']) with ordinality order by ordinality asc); +---- +{c1,a2,b3} + +query T +select array(select unnest || ordinality from unnest(array['c', 'a', 'b']) with ordinality order by ordinality desc); +---- +{b3,a2,c1} + +query T +select array(select unnest || ordinality from unnest(array['c', 'a', 'b']) with ordinality order by unnest asc); +---- +{a2,b3,c1} + +query T +select array(select unnest || ordinality from unnest(array['c', 'a', 'b']) with ordinality order by unnest desc); +---- +{c1,b3,a2} + +# Test order by a non-output column, and output column is not 0th of source table. +query T +select array(select ordinality from unnest(array['c', 'a', 'b']) with ordinality order by unnest desc); +---- +{1,3,2} + +# Test order by a non-output column, and order column is not 0th of source table. +query T +select array(select unnest from unnest(array['c', 'a', 'b']) with ordinality order by ordinality desc); +---- +{b,a,c} diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index dfcd32c9f4c78..b33f8ae31a6cc 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -589,7 +589,8 @@ impl TestCase { let mut logical_plan = match planner.plan(bound) { Ok(logical_plan) => { if self.expected_outputs.contains(&TestType::LogicalPlan) { - ret.logical_plan = Some(explain_plan(&logical_plan.clone().into_subplan())); + ret.logical_plan = + Some(explain_plan(&logical_plan.clone().into_unordered_subplan())); } logical_plan } diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 7988984bedb3d..5f32605b14184 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -45,6 +45,7 @@ pub use logical_optimization::*; pub use optimizer_context::*; use plan_expr_rewriter::ConstEvalRewriter; use property::Order; +use risingwave_common::bail; use risingwave_common::catalog::{ ColumnCatalog, ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, TableId, }; @@ -151,21 +152,48 @@ impl PlanRoot { } } - /// Get out fields of the plan root. - pub fn out_fields(&self) -> &FixedBitSet { - &self.out_fields - } - /// Transform the [`PlanRoot`] back to a [`PlanRef`] suitable to be used as a subplan, for /// example as insert source or subquery. This ignores Order but retains post-Order pruning /// (`out_fields`). - pub fn into_subplan(self) -> PlanRef { + pub fn into_unordered_subplan(self) -> PlanRef { if self.out_fields.count_ones(..) == self.out_fields.len() { return self.plan; } LogicalProject::with_out_fields(self.plan, &self.out_fields).into() } + /// Transform the [`PlanRoot`] wrapped in an array-construction subquery to a [`PlanRef`] + /// supported by `ARRAY_AGG`. Similar to the unordered version, this abstracts away internal + /// `self.plan` which is further modified by `self.required_order` then `self.out_fields`. + pub fn into_array_agg(self) -> Result { + use generic::Agg; + use plan_node::PlanAggCall; + use risingwave_common::types::DataType; + use risingwave_expr::aggregate::AggKind; + + use crate::expr::InputRef; + use crate::utils::{Condition, IndexSet}; + + let Ok(select_idx) = self.out_fields.ones().exactly_one() else { + bail!("subquery must return only one column"); + }; + let input_column_type = self.plan.schema().fields()[select_idx].data_type(); + Ok(Agg::new( + vec![PlanAggCall { + agg_kind: AggKind::ArrayAgg, + return_type: DataType::List(input_column_type.clone().into()), + inputs: vec![InputRef::new(select_idx, input_column_type)], + distinct: false, + order_by: self.required_order.column_orders, + filter: Condition::true_cond(), + direct_args: vec![], + }], + IndexSet::empty(), + self.plan, + ) + .into()) + } + /// Apply logical optimization to the plan for stream. pub fn gen_optimized_logical_plan_for_stream(&self) -> Result { LogicalOptimizer::gen_optimized_logical_plan_for_stream(self.plan.clone()) @@ -882,7 +910,7 @@ mod tests { out_fields, out_names, ); - let subplan = root.into_subplan(); + let subplan = root.into_unordered_subplan(); assert_eq!( subplan.schema(), &Schema::new(vec![Field::with_name(DataType::Int32, "v1")]) diff --git a/src/frontend/src/planner/insert.rs b/src/frontend/src/planner/insert.rs index 55ac8dffb991c..9189f17cb4d14 100644 --- a/src/frontend/src/planner/insert.rs +++ b/src/frontend/src/planner/insert.rs @@ -23,7 +23,7 @@ use crate::planner::Planner; impl Planner { pub(super) fn plan_insert(&mut self, insert: BoundInsert) -> Result { - let mut input = self.plan_query(insert.source)?.into_subplan(); + let mut input = self.plan_query(insert.source)?.into_unordered_subplan(); if !insert.cast_exprs.is_empty() { input = LogicalProject::create(input, insert.cast_exprs); } diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index ff51cb56ec9bf..42fdc83a3f933 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -42,7 +42,7 @@ impl Planner { Relation::BaseTable(t) => self.plan_base_table(&t), Relation::SystemTable(st) => self.plan_sys_table(*st), // TODO: order is ignored in the subquery - Relation::Subquery(q) => Ok(self.plan_query(q.query)?.into_subplan()), + Relation::Subquery(q) => Ok(self.plan_query(q.query)?.into_unordered_subplan()), Relation::Join(join) => self.plan_join(*join), Relation::Apply(join) => self.plan_apply(*join), Relation::WindowTableFunction(tf) => self.plan_window_table_function(*tf), diff --git a/src/frontend/src/planner/select.rs b/src/frontend/src/planner/select.rs index 3b603d7436a73..086ba88238d6f 100644 --- a/src/frontend/src/planner/select.rs +++ b/src/frontend/src/planner/select.rs @@ -15,13 +15,12 @@ use std::collections::HashMap; use itertools::Itertools; +use risingwave_common::bail_not_implemented; use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::ColumnOrder; -use risingwave_common::{bail, bail_not_implemented}; -use risingwave_expr::aggregate::AggKind; use risingwave_expr::ExprError; use risingwave_pb::plan_common::JoinType; @@ -219,30 +218,6 @@ impl Planner { Ok(LogicalProject::create(count_star.into(), vec![ge.into()])) } - /// Helper to create an `ARRAY_AGG` operator with the given `input`. - /// It is represented by `ARRAY_AGG($0) -> input` - fn create_array_agg(&self, input: PlanRef) -> Result { - let fields = input.schema().fields(); - if fields.len() != 1 { - bail!("subquery must return only one column"); - } - let input_column_type = fields[0].data_type(); - Ok(Agg::new( - vec![PlanAggCall { - agg_kind: AggKind::ArrayAgg, - return_type: DataType::List(input.schema().fields()[0].data_type().into()), - inputs: vec![InputRef::new(0, input_column_type)], - distinct: false, - order_by: vec![], - filter: Condition::true_cond(), - direct_args: vec![], - }], - IndexSet::empty(), - input, - ) - .into()) - } - /// For `(NOT) EXISTS subquery` or `(NOT) IN subquery`, we can plan it as /// `LeftSemi/LeftAnti` [`LogicalApply`] /// For other subqueries, we plan it as `LeftOuter` [`LogicalApply`] using @@ -316,7 +291,7 @@ impl Planner { let correlated_indices = subquery.collect_correlated_indices_by_depth_and_assign_id(0, correlated_id); let output_column_type = subquery.query.data_types()[0].clone(); - let right_plan = self.plan_query(subquery.query)?.into_subplan(); + let right_plan = self.plan_query(subquery.query)?.into_unordered_subplan(); let on = match subquery.kind { SubqueryKind::Existential => ExprImpl::literal_bool(true), SubqueryKind::In(left_expr) => { @@ -391,18 +366,16 @@ impl Planner { .zip_eq_fast(rewriter.correlated_indices_collection) .zip_eq_fast(rewriter.correlated_ids) { - let mut right = self.plan_query(subquery.query)?.into_subplan(); + let subroot = self.plan_query(subquery.query)?; - match subquery.kind { - SubqueryKind::Scalar => {} + let right = match subquery.kind { + SubqueryKind::Scalar => subroot.into_unordered_subplan(), SubqueryKind::Existential => { - right = self.create_exists(right)?; - } - SubqueryKind::Array => { - right = self.create_array_agg(right)?; + self.create_exists(subroot.into_unordered_subplan())? } + SubqueryKind::Array => subroot.into_array_agg()?, _ => bail_not_implemented!(issue = 1343, "{:?}", subquery.kind), - } + }; root = Self::create_apply( correlated_id, diff --git a/src/frontend/src/planner/set_expr.rs b/src/frontend/src/planner/set_expr.rs index eea05024db540..735712c9b055e 100644 --- a/src/frontend/src/planner/set_expr.rs +++ b/src/frontend/src/planner/set_expr.rs @@ -30,7 +30,7 @@ impl Planner { match set_expr { BoundSetExpr::Select(s) => self.plan_select(*s, extra_order_exprs, order), BoundSetExpr::Values(v) => self.plan_values(*v), - BoundSetExpr::Query(q) => Ok(self.plan_query(*q)?.into_subplan()), + BoundSetExpr::Query(q) => Ok(self.plan_query(*q)?.into_unordered_subplan()), BoundSetExpr::SetOperation { op, all, From d3fb88dadf5c92fa7e25b74f12ec6d8731dc1763 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 10 Jan 2024 13:29:37 +0800 Subject: [PATCH 08/17] chore(ci): fix `backfill-tests` (#14472) --- ci/scripts/run-backfill-tests.sh | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/ci/scripts/run-backfill-tests.sh b/ci/scripts/run-backfill-tests.sh index dddf88e4b4cac..46bad790d21f7 100755 --- a/ci/scripts/run-backfill-tests.sh +++ b/ci/scripts/run-backfill-tests.sh @@ -168,6 +168,7 @@ test_replication_with_column_pruning() { echo "--- Kill cluster" cargo make kill cargo make wait-processes-exit + wait } # Test sink backfill recovery @@ -177,16 +178,11 @@ test_sink_backfill_recovery() { # Check progress sqllogictest -p 4566 -d dev 'e2e_test/backfill/sink/create_sink.slt' - sqllogictest -p 4566 -d dev 'e2e_test/background_ddl/common/validate_one_job.slt' # Restart restart_cluster sleep 3 - # FIXME(kwannoel): Sink's backfill progress is not recovered yet. - # Check progress - # sqllogictest -p 4566 -d dev 'e2e_test/background_ddl/common/validate_one_job.slt' - # Sink back into rw run_sql "CREATE TABLE table_kafka (v1 int primary key) WITH ( @@ -199,6 +195,9 @@ test_sink_backfill_recovery() { # Verify data matches upstream table. sqllogictest -p 4566 -d dev 'e2e_test/backfill/sink/validate_sink.slt' + cargo make kill + cargo make wait-processes-exit + wait } main() { From 7a8133a39fbb30c4859646dd5331905cc9b87368 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Wed, 10 Jan 2024 13:58:32 +0800 Subject: [PATCH 09/17] feat(dashboard): display back-pressure on fragment graph (#14446) Signed-off-by: Bugen Zhao --- dashboard/components/BackPressureTable.tsx | 12 +- dashboard/components/FragmentGraph.tsx | 143 +++++++++++++++++---- dashboard/hook/useErrorToast.ts | 2 - dashboard/lib/layout.ts | 29 ++++- dashboard/pages/api/api.ts | 8 ++ dashboard/pages/api/fetch.ts | 34 ++++- dashboard/pages/api/metric.ts | 10 +- dashboard/pages/fragment_graph.tsx | 95 +++++++++++++- src/meta/src/dashboard/mod.rs | 4 +- 9 files changed, 283 insertions(+), 54 deletions(-) diff --git a/dashboard/components/BackPressureTable.tsx b/dashboard/components/BackPressureTable.tsx index 241fde0ddc13c..ad790c0cb680a 100644 --- a/dashboard/components/BackPressureTable.tsx +++ b/dashboard/components/BackPressureTable.tsx @@ -29,13 +29,11 @@ import { sortBy } from "lodash" import Head from "next/head" import { Fragment, useEffect, useState } from "react" import useErrorToast from "../hook/useErrorToast" -import { getActorBackPressures } from "../pages/api/metric" +import { + BackPressuresMetrics, + getActorBackPressures, +} from "../pages/api/metric" import RateBar from "./RateBar" -import { Metrics } from "./metrics" - -interface BackPressuresMetrics { - outputBufferBlockingDuration: Metrics[] -} export default function BackPressureTable({ selectedFragmentIds, @@ -50,7 +48,7 @@ export default function BackPressureTable({ async function doFetch() { while (true) { try { - let metrics: BackPressuresMetrics = await getActorBackPressures() + let metrics = await getActorBackPressures() metrics.outputBufferBlockingDuration = sortBy( metrics.outputBufferBlockingDuration, (m) => (m.metric.fragment_id, m.metric.downstream_fragment_id) diff --git a/dashboard/components/FragmentGraph.tsx b/dashboard/components/FragmentGraph.tsx index d9ca56fea69e8..81d8695d0c31c 100644 --- a/dashboard/components/FragmentGraph.tsx +++ b/dashboard/components/FragmentGraph.tsx @@ -10,11 +10,13 @@ import { theme, useDisclosure, } from "@chakra-ui/react" +import { tinycolor } from "@ctrl/tinycolor" import loadable from "@loadable/component" import * as d3 from "d3" import { cloneDeep } from "lodash" import { Fragment, useCallback, useEffect, useRef, useState } from "react" import { + Edge, FragmentBox, FragmentBoxPosition, Position, @@ -22,7 +24,6 @@ import { layout, } from "../lib/layout" import { PlanNodeDatum } from "../pages/fragment_graph" -import BackPressureTable from "./BackPressureTable" const ReactJson = loadable(() => import("react-json-view")) @@ -86,19 +87,21 @@ function boundBox( const nodeRadius = 12 const nodeMarginX = nodeRadius * 6 const nodeMarginY = nodeRadius * 4 -const fragmentMarginX = nodeRadius -const fragmentMarginY = nodeRadius -const fragmentDistanceX = nodeRadius * 5 -const fragmentDistanceY = nodeRadius * 5 +const fragmentMarginX = nodeRadius * 2 +const fragmentMarginY = nodeRadius * 2 +const fragmentDistanceX = nodeRadius * 2 +const fragmentDistanceY = nodeRadius * 2 export default function FragmentGraph({ planNodeDependencies, fragmentDependency, selectedFragmentId, + backPressures, }: { planNodeDependencies: Map> fragmentDependency: FragmentBox[] - selectedFragmentId: string | undefined + selectedFragmentId?: string + backPressures?: Map }) { const svgRef = useRef(null) @@ -288,6 +291,7 @@ export default function FragmentGraph({ const applyStreamNode = (g: StreamNodeSelection) => { g.attr("transform", (d) => `translate(${d.x},${d.y})`) + // Node circle let circle = g.select("circle") if (circle.empty()) { circle = g.append("circle") @@ -299,6 +303,7 @@ export default function FragmentGraph({ .style("cursor", "pointer") .on("click", (_d, i) => openPlanNodeDetail(i.data)) + // Node name under the circle let text = g.select("text") if (text.empty()) { text = g.append("text") @@ -330,13 +335,8 @@ export default function FragmentGraph({ streamNodeSelection.call(applyStreamNode) } - const createFragment = (sel: Enter) => { - const gSel = sel - .append("g") - .attr("class", "fragment") - .call(applyFragment) - return gSel - } + const createFragment = (sel: Enter) => + sel.append("g").attr("class", "fragment").call(applyFragment) const fragmentSelection = svgSelection .select(".fragments") @@ -351,7 +351,7 @@ export default function FragmentGraph({ // Fragment Edges const edgeSelection = svgSelection .select(".fragment-edges") - .selectAll(".fragment-edge") + .selectAll(".fragment-edge") .data(fragmentEdgeLayout) type EdgeSelection = typeof edgeSelection @@ -363,20 +363,69 @@ export default function FragmentGraph({ .x(({ x }) => x) .y(({ y }) => y) - const applyEdge = (sel: EdgeSelection) => - sel + const applyEdge = (gSel: EdgeSelection) => { + // Edge line + let path = gSel.select("path") + if (path.empty()) { + path = gSel.append("path") + } + + const isEdgeSelected = (d: Edge) => + isSelected(d.source) || isSelected(d.target) + + const color = (d: Edge) => { + if (backPressures) { + let value = backPressures.get(`${d.target}_${d.source}`) + if (value) { + return backPressureColor(value) + } + } + + return isEdgeSelected(d) + ? theme.colors.blue["500"] + : theme.colors.gray["300"] + } + + const width = (d: Edge) => { + if (backPressures) { + let value = backPressures.get(`${d.target}_${d.source}`) + if (value) { + return backPressureWidth(value) + } + } + + return isEdgeSelected(d) ? 4 : 2 + } + + path .attr("d", ({ points }) => line(points)) .attr("fill", "none") - .attr("stroke-width", (d) => - isSelected(d.source) || isSelected(d.target) ? 2 : 1 - ) - .attr("stroke", (d) => - isSelected(d.source) || isSelected(d.target) - ? theme.colors.blue["500"] - : theme.colors.gray["300"] - ) + .attr("stroke-width", width) + .attr("stroke", color) + + // Tooltip for back pressure rate + let title = gSel.select("title") + if (title.empty()) { + title = gSel.append("title") + } + + const text = (d: Edge) => { + if (backPressures) { + let value = backPressures.get(`${d.target}_${d.source}`) + if (value) { + return `${value.toFixed(2)}%` + } + } + + return "" + } + + title.text(text) + + return gSel + } const createEdge = (sel: Enter) => - sel.append("path").attr("class", "fragment-edge").call(applyEdge) + sel.append("g").attr("class", "fragment-edge").call(applyEdge) edgeSelection.enter().call(createEdge) edgeSelection.call(applyEdge) @@ -385,6 +434,7 @@ export default function FragmentGraph({ }, [ fragmentLayout, fragmentEdgeLayout, + backPressures, selectedFragmentId, openPlanNodeDetail, ]) @@ -423,7 +473,48 @@ export default function FragmentGraph({ - + {/* */} ) } + +/** + * The color for the edge with given back pressure value. + * + * @param value The back pressure rate, between 0 and 100. + */ +function backPressureColor(value: number) { + const colorRange = [ + theme.colors.green["100"], + theme.colors.green["300"], + theme.colors.yellow["400"], + theme.colors.orange["500"], + theme.colors.red["700"], + ].map((c) => tinycolor(c)) + + value = Math.max(value, 0) + value = Math.min(value, 100) + + const step = colorRange.length - 1 + const pos = (value / 100) * step + const floor = Math.floor(pos) + const ceil = Math.ceil(pos) + + const color = tinycolor(colorRange[floor]) + .mix(tinycolor(colorRange[ceil]), (pos - floor) * 100) + .toHexString() + + return color +} + +/** + * The width for the edge with given back pressure value. + * + * @param value The back pressure rate, between 0 and 100. + */ +function backPressureWidth(value: number) { + value = Math.max(value, 0) + value = Math.min(value, 100) + + return 30 * (value / 100) + 2 +} diff --git a/dashboard/hook/useErrorToast.ts b/dashboard/hook/useErrorToast.ts index 1c86cc3084fbf..1b4f0f80606cb 100644 --- a/dashboard/hook/useErrorToast.ts +++ b/dashboard/hook/useErrorToast.ts @@ -39,8 +39,6 @@ export default function useErrorToast() { duration: 5000, isClosable: true, }) - - console.error(e) }, [toast] ) diff --git a/dashboard/lib/layout.ts b/dashboard/lib/layout.ts index c0e61faeddbf1..1182976dfe8cb 100644 --- a/dashboard/lib/layout.ts +++ b/dashboard/lib/layout.ts @@ -275,6 +275,7 @@ export interface FragmentBox { width: number height: number parentIds: string[] + externalParentIds: string[] fragment?: TableFragments_Fragment } @@ -414,14 +415,11 @@ export function layoutPoint( nodeRadius: number ): FragmentPointPosition[] { const fragmentBoxes: Array = [] - for (let { id, name, order, parentIds, ...others } of fragments) { + for (let { ...others } of fragments) { fragmentBoxes.push({ - id, - name, - parentIds, width: nodeRadius * 2, height: nodeRadius * 2, - order, + externalParentIds: [], // we don't care about external parent for point layout ...others, }) } @@ -498,6 +496,27 @@ export function generateBoxEdges(layoutMap: FragmentBoxPosition[]): Edge[] { target: parentId, }) } + + // Simply draw a horizontal line here. + // Typically, external parent is only applicable to `StreamScan` fragment, + // and there'll be only one external parent due to `UpstreamShard` distribution + // and plan node sharing. So there's no overlapping issue. + for (const externalParentId of fragment.externalParentIds) { + links.push({ + points: [ + { + x: fragment.x, + y: fragment.y + fragment.height / 2, + }, + { + x: fragment.x + 100, + y: fragment.y + fragment.height / 2, + }, + ], + source: fragment.id, + target: externalParentId, + }) + } } return links } diff --git a/dashboard/pages/api/api.ts b/dashboard/pages/api/api.ts index aa63e28eb67f1..0df0454295482 100644 --- a/dashboard/pages/api/api.ts +++ b/dashboard/pages/api/api.ts @@ -46,6 +46,14 @@ class Api { try { const res = await fetch(url) const data = await res.json() + + // Throw error if response is not ok. + // See `DashboardError::into_response`. + if (!res.ok) { + throw `${res.status} ${res.statusText}${ + data.error ? ": " + data.error : "" + }` + } return data } catch (e) { console.error(e) diff --git a/dashboard/pages/api/fetch.ts b/dashboard/pages/api/fetch.ts index b545ca60dee25..7aa34826e6a5b 100644 --- a/dashboard/pages/api/fetch.ts +++ b/dashboard/pages/api/fetch.ts @@ -18,21 +18,41 @@ import { useEffect, useState } from "react" import useErrorToast from "../../hook/useErrorToast" -export default function useFetch(fetchFn: () => Promise) { +/** + * Fetch data from the server and return the response. + * @param fetchFn The function to fetch data from the server. + * @param intervalMs The interval in milliseconds to fetch data from the server. If null, the data is fetched only once. + * @param when If true, fetch data from the server. If false, do nothing. + * @returns The response from the server. + */ +export default function useFetch( + fetchFn: () => Promise, + intervalMs: number | null = null, + when: boolean = true +) { const [response, setResponse] = useState() const toast = useErrorToast() useEffect(() => { const fetchData = async () => { - try { - const res = await fetchFn() - setResponse(res) - } catch (e: any) { - toast(e) + if (when) { + try { + const res = await fetchFn() + setResponse(res) + } catch (e: any) { + toast(e) + } } } fetchData() - }, [toast, fetchFn]) + + if (!intervalMs) { + return + } + + const timer = setInterval(fetchData, intervalMs) + return () => clearInterval(timer) + }, [toast, fetchFn, intervalMs, when]) return { response } } diff --git a/dashboard/pages/api/metric.ts b/dashboard/pages/api/metric.ts index c799e52e71a70..2ea62cebc4385 100644 --- a/dashboard/pages/api/metric.ts +++ b/dashboard/pages/api/metric.ts @@ -14,11 +14,17 @@ * limitations under the License. * */ -import { MetricsSample } from "../../components/metrics" +import { Metrics, MetricsSample } from "../../components/metrics" import api from "./api" +export interface BackPressuresMetrics { + outputBufferBlockingDuration: Metrics[] +} + export async function getActorBackPressures() { - const res = await api.get("/metrics/actor/back_pressures") + const res: BackPressuresMetrics = await api.get( + "/metrics/actor/back_pressures" + ) return res } diff --git a/dashboard/pages/fragment_graph.tsx b/dashboard/pages/fragment_graph.tsx index 18042c6450dc3..db1cb72ca2122 100644 --- a/dashboard/pages/fragment_graph.tsx +++ b/dashboard/pages/fragment_graph.tsx @@ -32,15 +32,16 @@ import { dagStratify } from "d3-dag" import _ from "lodash" import Head from "next/head" import { useRouter } from "next/router" -import { Fragment, useCallback, useEffect, useState } from "react" +import { Fragment, useCallback, useEffect, useMemo, useState } from "react" import FragmentDependencyGraph from "../components/FragmentDependencyGraph" import FragmentGraph from "../components/FragmentGraph" import Title from "../components/Title" import useErrorToast from "../hook/useErrorToast" import { FragmentBox } from "../lib/layout" import { TableFragments, TableFragments_Fragment } from "../proto/gen/meta" -import { Dispatcher, StreamNode } from "../proto/gen/stream_plan" +import { Dispatcher, MergeNode, StreamNode } from "../proto/gen/stream_plan" import useFetch from "./api/fetch" +import { getActorBackPressures, p50, p90, p95, p99 } from "./api/metric" import { getFragments, getStreamingJobs } from "./api/streaming" interface DispatcherNode { @@ -101,6 +102,22 @@ function buildPlanNodeDependency( }) } +function findMergeNodes(root: StreamNode): MergeNode[] { + let mergeNodes = new Set() + + const findMergeNodesRecursive = (node: StreamNode) => { + if (node.nodeBody?.$case === "merge") { + mergeNodes.add(node.nodeBody.merge) + } + for (const child of node.input || []) { + findMergeNodesRecursive(child) + } + } + + findMergeNodesRecursive(root) + return Array.from(mergeNodes) +} + function buildFragmentDependencyAsEdges( fragments: TableFragments ): FragmentBox[] { @@ -115,11 +132,17 @@ function buildFragmentDependencyAsEdges( for (const id in fragments.fragments) { const fragment = fragments.fragments[id] const parentIds = new Set() + const externalParentIds = new Set() + for (const actor of fragment.actors) { for (const upstreamActorId of actor.upstreamActorId) { const upstreamFragmentId = actorToFragmentMapping.get(upstreamActorId) if (upstreamFragmentId) { parentIds.add(upstreamFragmentId) + } else { + for (const m of findMergeNodes(actor.nodes!)) { + externalParentIds.add(m.upstreamFragmentId) + } } } } @@ -127,24 +150,35 @@ function buildFragmentDependencyAsEdges( id: fragment.fragmentId.toString(), name: `Fragment ${fragment.fragmentId}`, parentIds: Array.from(parentIds).map((x) => x.toString()), + externalParentIds: Array.from(externalParentIds).map((x) => x.toString()), width: 0, height: 0, order: fragment.fragmentId, fragment, - } as FragmentBox) + }) } return nodes } const SIDEBAR_WIDTH = 200 +type BackPressureAlgo = "p50" | "p90" | "p95" | "p99" +const backPressureAlgos: BackPressureAlgo[] = ["p50", "p90", "p95", "p99"] + export default function Streaming() { const { response: relationList } = useFetch(getStreamingJobs) const { response: fragmentList } = useFetch(getFragments) const [selectedFragmentId, setSelectedFragmentId] = useState() + const [backPressureAlgo, setBackPressureAlgo] = useState() const router = useRouter() + const { response: actorBackPressures } = useFetch( + getActorBackPressures, + 5000, + backPressureAlgo !== undefined + ) + const fragmentDependencyCallback = useCallback(() => { if (fragmentList) { if (router.query.id) { @@ -256,6 +290,40 @@ export default function Streaming() { toast(new Error(`Actor ${searchActorIdInt} not found`)) } + const backPressures = useMemo(() => { + if (actorBackPressures && backPressureAlgo) { + let map = new Map() + + for (const m of actorBackPressures.outputBufferBlockingDuration) { + console.log(backPressureAlgo) + let algoFunc + switch (backPressureAlgo) { + case "p50": + algoFunc = p50 + break + case "p90": + algoFunc = p90 + break + case "p95": + algoFunc = p95 + break + case "p99": + algoFunc = p99 + break + default: + return + } + + const value = algoFunc(m.sample) * 100 + map.set( + `${m.metric.fragment_id}_${m.metric.downstream_fragment_id}`, + value + ) + } + return map + } + }, [actorBackPressures, backPressureAlgo]) + const retVal = ( Fragment Graph @@ -324,6 +392,26 @@ export default function Streaming() { + + Back Pressure + + Fragments {fragmentDependencyDag && ( @@ -353,6 +441,7 @@ export default function Streaming() { selectedFragmentId={selectedFragmentId?.toString()} fragmentDependency={fragmentDependency} planNodeDependencies={planNodeDependencies} + backPressures={backPressures} /> )} diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index 98b5c11c62f77..c1cdb5ef1ec19 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -66,6 +66,7 @@ pub(super) mod handlers { HeapProfilingResponse, ListHeapProfilingResponse, StackTraceResponse, }; use serde_json::json; + use thiserror_ext::AsReport; use super::*; use crate::manager::WorkerId; @@ -87,8 +88,7 @@ pub(super) mod handlers { impl IntoResponse for DashboardError { fn into_response(self) -> axum::response::Response { let mut resp = Json(json!({ - "error": format!("{}", self.0), - "info": format!("{:?}", self.0), + "error": self.0.to_report_string(), })) .into_response(); *resp.status_mut() = StatusCode::INTERNAL_SERVER_ERROR; From 0ace94e61e8a90d7cca84267b295d8be424a6aa5 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 10 Jan 2024 14:37:59 +0800 Subject: [PATCH 10/17] refactor: local barrier manager use prev epoch to track barrier state (#14436) --- src/stream/src/task/barrier_manager.rs | 90 +++--- .../src/task/barrier_manager/managed_state.rs | 284 +++++++++--------- src/stream/src/task/barrier_manager/tests.rs | 15 +- src/stream/src/task/stream_manager.rs | 1 - 4 files changed, 204 insertions(+), 186 deletions(-) diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 0a8eede9e172a..b706d7a5c3537 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -16,15 +16,14 @@ use std::collections::{HashMap, HashSet}; use std::sync::Arc; use anyhow::anyhow; -use prometheus::HistogramTimer; -use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_pb::stream_service::barrier_complete_response::PbCreateMviewProgress; +use thiserror_ext::AsReport; use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender}; use tokio::sync::oneshot; use tokio::sync::oneshot::Receiver; use self::managed_state::ManagedBarrierState; -use crate::error::{StreamError, StreamResult}; +use crate::error::{IntoUnexpectedExit, StreamError, StreamResult}; use crate::task::ActorId; mod managed_state; @@ -33,6 +32,7 @@ mod progress; mod tests; pub use progress::CreateMviewProgress; +use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_storage::StateStoreImpl; use crate::executor::monitor::StreamingMetrics; @@ -61,8 +61,8 @@ enum LocalBarrierEvent { }, InjectBarrier { barrier: Barrier, - actor_ids_to_send: Vec, - actor_ids_to_collect: Vec, + actor_ids_to_send: HashSet, + actor_ids_to_collect: HashSet, result_sender: oneshot::Sender>, }, Reset, @@ -92,7 +92,7 @@ enum LocalBarrierEvent { /// barriers to and collect them from all actors, and finally report the progress. struct LocalBarrierWorker { /// Stores all streaming job source sender. - senders: HashMap>>, + barrier_senders: HashMap>>, /// Current barrier collection state. state: ManagedBarrierState, @@ -100,15 +100,14 @@ struct LocalBarrierWorker { /// Save collect `CompleteReceiver`. collect_complete_receiver: HashMap, - streaming_metrics: Arc, + /// Record all unexpected exited actors. + failure_actors: HashMap, } /// Information used after collection. pub struct CompleteReceiver { /// Notify all actors of completion of collection. pub complete_receiver: Option>>, - /// `barrier_inflight_timer`'s metrics. - pub barrier_inflight_timer: HistogramTimer, /// The kind of barrier. pub kind: BarrierKind, } @@ -116,10 +115,10 @@ pub struct CompleteReceiver { impl LocalBarrierWorker { fn new(state_store: StateStoreImpl, streaming_metrics: Arc) -> Self { Self { - senders: HashMap::new(), - state: ManagedBarrierState::new(state_store), + barrier_senders: HashMap::new(), + failure_actors: HashMap::default(), + state: ManagedBarrierState::new(state_store, streaming_metrics), collect_complete_receiver: HashMap::default(), - streaming_metrics, } } @@ -135,12 +134,8 @@ impl LocalBarrierWorker { actor_ids_to_collect, result_sender, } => { - let timer = self - .streaming_metrics - .barrier_inflight_latency - .start_timer(); let result = - self.send_barrier(&barrier, actor_ids_to_send, actor_ids_to_collect, timer); + self.send_barrier(&barrier, actor_ids_to_send, actor_ids_to_collect); let _ = result_sender.send(result).inspect_err(|e| { warn!(err=?e, "fail to send inject barrier result"); }); @@ -184,7 +179,10 @@ impl LocalBarrierWorker { actor_id = actor_id, "register sender" ); - self.senders.entry(actor_id).or_default().push(sender); + self.barrier_senders + .entry(actor_id) + .or_default() + .push(sender); } /// Broadcast a barrier to all senders. Save a receiver which will get notified when this @@ -192,12 +190,9 @@ impl LocalBarrierWorker { fn send_barrier( &mut self, barrier: &Barrier, - actor_ids_to_send: impl IntoIterator, - actor_ids_to_collect: impl IntoIterator, - timer: HistogramTimer, + to_send: HashSet, + to_collect: HashSet, ) -> StreamResult<()> { - let to_send: HashSet = actor_ids_to_send.into_iter().collect(); - let to_collect: HashSet = actor_ids_to_collect.into_iter().collect(); debug!( target: "events::stream::barrier::manager::send", "send barrier {:?}, senders = {:?}, actor_ids_to_collect = {:?}", @@ -209,11 +204,20 @@ impl LocalBarrierWorker { // There must be some actors to collect from. assert!(!to_collect.is_empty()); + for actor_id in &to_collect { + if let Some(e) = self.failure_actors.get(actor_id) { + // The failure actors could exit before the barrier is issued, while their + // up-downstream actors could be stuck somehow. Return error directly to trigger the + // recovery. + return Err(e.clone()); + } + } + let (tx, rx) = oneshot::channel(); - self.state.transform_to_issued(barrier, to_collect, tx)?; + self.state.transform_to_issued(barrier, to_collect, tx); for actor_id in to_send { - match self.senders.get(&actor_id) { + match self.barrier_senders.get(&actor_id) { Some(senders) => { for sender in senders { if let Err(_err) = sender.send(barrier.clone()) { @@ -244,7 +248,7 @@ impl LocalBarrierWorker { actors ); for actor in actors { - self.senders.remove(actor); + self.barrier_senders.remove(actor); } } @@ -252,7 +256,6 @@ impl LocalBarrierWorker { barrier.epoch.prev, CompleteReceiver { complete_receiver: Some(rx), - barrier_inflight_timer: timer, kind: barrier.kind, }, ); @@ -277,10 +280,10 @@ impl LocalBarrierWorker { /// Reset all internal states. fn reset(&mut self) { - self.senders.clear(); - self.collect_complete_receiver.clear(); - - self.state.clear_all_states(); + *self = Self::new( + self.state.state_store.clone(), + self.state.streaming_metrics.clone(), + ); } /// When a [`crate::executor::StreamConsumer`] (typically [`crate::executor::DispatchExecutor`]) get a barrier, it should report @@ -292,7 +295,24 @@ impl LocalBarrierWorker { /// When a actor exit unexpectedly, it should report this event using this function, so meta /// will notice actor's exit while collecting. fn notify_failure(&mut self, actor_id: ActorId, err: StreamError) { - self.state.notify_failure(actor_id, err) + let err = err.into_unexpected_exit(actor_id); + if let Some(prev_err) = self.failure_actors.insert(actor_id, err.clone()) { + warn!( + actor_id, + prev_err = %prev_err.as_report(), + "actor error overwritten" + ); + } + for (fail_epoch, notifier) in self.state.notifiers_await_on_actor(actor_id) { + if notifier.send(Err(err.clone())).is_err() { + warn!( + fail_epoch, + actor_id, + err = %err.as_report(), + "fail to notify actor failure" + ); + } + } } } @@ -340,7 +360,8 @@ impl LocalBarrierManager { actor_ids_to_collect: actor_ids_to_collect.into_iter().collect(), result_sender: tx, }); - rx.await.expect("should receive response") + rx.await + .map_err(|_| anyhow!("barrier manager maybe reset"))? } /// Use `prev_epoch` to remove collect rx and return rx. @@ -350,7 +371,8 @@ impl LocalBarrierManager { epoch: prev_epoch, result_sender: tx, }); - rx.await.expect("should receive response") + rx.await + .map_err(|_| anyhow!("barrier manager maybe reset"))? } /// Reset all internal states. diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 6c31a37f6edc0..9418d4e8abcc6 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -14,17 +14,19 @@ use std::collections::{BTreeMap, HashMap, HashSet}; use std::iter::once; +use std::ops::Sub; +use std::sync::Arc; +use prometheus::HistogramTimer; use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; use risingwave_storage::{dispatch_state_store, StateStore, StateStoreImpl}; -use thiserror_ext::AsReport; use tokio::sync::oneshot; use super::progress::BackfillState; use super::CollectResult; -use crate::error::{IntoUnexpectedExit, StreamError, StreamResult}; -use crate::executor::monitor::GLOBAL_STREAMING_METRICS; +use crate::error::StreamResult; +use crate::executor::monitor::StreamingMetrics; use crate::executor::Barrier; use crate::task::ActorId; @@ -45,50 +47,59 @@ enum ManagedBarrierStateInner { /// Notify that the collection is finished. collect_notifier: Option>>, + + barrier_inflight_latency: HistogramTimer, }, } #[derive(Debug)] pub(super) struct BarrierState { - prev_epoch: u64, + curr_epoch: u64, inner: ManagedBarrierStateInner, kind: BarrierKind, } -#[derive(Debug)] pub(super) struct ManagedBarrierState { /// Record barrier state for each epoch of concurrent checkpoints. /// - /// The key is curr_epoch, and the first value is prev_epoch + /// The key is prev_epoch, and the first value is curr_epoch epoch_barrier_state_map: BTreeMap, /// Record the progress updates of creating mviews for each epoch of concurrent checkpoints. pub(super) create_mview_progress: HashMap>, - /// Record all unexpected exited actors. - failure_actors: HashMap, + pub(super) state_store: StateStoreImpl, - state_store: StateStoreImpl, + pub(super) streaming_metrics: Arc, } impl ManagedBarrierState { + #[cfg(test)] + pub(crate) fn for_test() -> Self { + Self::new( + StateStoreImpl::for_test(), + Arc::new(StreamingMetrics::unused()), + ) + } + /// Create a barrier manager state. This will be called only once. - pub(super) fn new(state_store: StateStoreImpl) -> Self { + pub(super) fn new( + state_store: StateStoreImpl, + streaming_metrics: Arc, + ) -> Self { Self { epoch_barrier_state_map: BTreeMap::default(), create_mview_progress: Default::default(), - failure_actors: Default::default(), state_store, + streaming_metrics, } } /// Notify if we have collected barriers from all actor ids. The state must be `Issued`. - fn may_notify(&mut self, curr_epoch: u64) { + fn may_notify(&mut self, prev_epoch: u64) { // Report if there's progress on the earliest in-flight barrier. - if self.epoch_barrier_state_map.keys().next() == Some(&curr_epoch) { - if let Some(metrics) = GLOBAL_STREAMING_METRICS.get() { - metrics.barrier_manager_progress.inc(); - } + if self.epoch_barrier_state_map.keys().next() == Some(&prev_epoch) { + self.streaming_metrics.barrier_manager_progress.inc(); } while let Some(entry) = self.epoch_barrier_state_map.first_entry() { @@ -103,10 +114,23 @@ impl ManagedBarrierState { break; } - let (epoch, barrier_state) = entry.remove_entry(); + let (prev_epoch, barrier_state) = entry.remove_entry(); + + let collect_notifier = match barrier_state.inner { + ManagedBarrierStateInner::Issued { + collect_notifier, + barrier_inflight_latency: timer, + .. + } => { + timer.observe_duration(); + collect_notifier + } + _ => unreachable!(), + }; + let create_mview_progress = self .create_mview_progress - .remove(&epoch) + .remove(&barrier_state.curr_epoch) .unwrap_or_default() .into_iter() .map(|(actor, state)| CreateMviewProgress { @@ -114,7 +138,7 @@ impl ManagedBarrierState { done: matches!(state, BackfillState::Done(_)), consumed_epoch: match state { BackfillState::ConsumingUpstream(consumed_epoch, _) => consumed_epoch, - BackfillState::Done(_) => epoch, + BackfillState::Done(_) => barrier_state.curr_epoch, }, consumed_rows: match state { BackfillState::ConsumingUpstream(_, consumed_rows) => consumed_rows, @@ -127,77 +151,76 @@ impl ManagedBarrierState { match kind { BarrierKind::Unspecified => unreachable!(), BarrierKind::Initial => tracing::info!( - epoch = barrier_state.prev_epoch, + epoch = prev_epoch, "ignore sealing data for the first barrier" ), BarrierKind::Barrier | BarrierKind::Checkpoint => { dispatch_state_store!(&self.state_store, state_store, { - state_store.seal_epoch(barrier_state.prev_epoch, kind.is_checkpoint()); + state_store.seal_epoch(prev_epoch, kind.is_checkpoint()); }); } } - match barrier_state.inner { - ManagedBarrierStateInner::Issued { - collect_notifier, .. - } => { - // Notify about barrier finishing. - let result = CollectResult { - create_mview_progress, - kind, - }; - if collect_notifier.unwrap().send(Ok(result)).is_err() { - warn!("failed to notify barrier collection with epoch {}", epoch) - } + if let Some(notifier) = collect_notifier { + // Notify about barrier finishing. + let result = CollectResult { + create_mview_progress, + kind, + }; + + if notifier.send(Ok(result)).is_err() { + warn!( + "failed to notify barrier collection with epoch {}", + prev_epoch + ) } - _ => unreachable!(), } } } - /// Clear and reset all states. - pub(crate) fn clear_all_states(&mut self) { - tracing::debug!("clear all states in local barrier manager"); - - *self = Self::new(self.state_store.clone()); - } - - /// Notify unexpected actor exit with given `actor_id`. - pub(crate) fn notify_failure(&mut self, actor_id: ActorId, err: StreamError) { - // Attach the actor id to the error. - let err = err.into_unexpected_exit(actor_id); - - for barrier_state in self.epoch_barrier_state_map.values_mut() { - #[allow(clippy::single_match)] - match barrier_state.inner { - ManagedBarrierStateInner::Issued { - ref remaining_actors, - ref mut collect_notifier, - } => { - if remaining_actors.contains(&actor_id) - && let Some(collect_notifier) = collect_notifier.take() - && collect_notifier.send(Err(err.clone())).is_err() - { - warn!(error = %err.as_report(), actor_id, "failed to notify actor exiting"); + /// Returns an iterator on the notifiers of epochs that is awaiting on `actor_id`. + /// This is used on notifying actor failure. On actor failure, the + /// barrier manager can call this method to iterate on notifiers of epochs that + /// waits on the failed actor and then notify failure on the result + /// sender of the epoch. + pub(crate) fn notifiers_await_on_actor( + &mut self, + actor_id: ActorId, + ) -> impl Iterator>)> + '_ { + self.epoch_barrier_state_map + .iter_mut() + .filter_map(move |(prev_epoch, barrier_state)| { + #[allow(clippy::single_match)] + match &mut barrier_state.inner { + ManagedBarrierStateInner::Issued { + ref remaining_actors, + ref mut collect_notifier, + .. + } => { + if remaining_actors.contains(&actor_id) { + collect_notifier + .take() + .map(|notifier| (*prev_epoch, notifier)) + } else { + None + } } + _ => None, } - _ => {} - } - } - self.failure_actors.insert(actor_id, err); + }) } /// Collect a `barrier` from the actor with `actor_id`. pub(super) fn collect(&mut self, actor_id: ActorId, barrier: &Barrier) { tracing::debug!( target: "events::stream::barrier::manager::collect", - epoch = barrier.epoch.curr, actor_id, state = ?self, + epoch = ?barrier.epoch, actor_id, state = ?self.epoch_barrier_state_map, "collect_barrier", ); - match self.epoch_barrier_state_map.get_mut(&barrier.epoch.curr) { + match self.epoch_barrier_state_map.get_mut(&barrier.epoch.prev) { Some(&mut BarrierState { - prev_epoch, + curr_epoch, inner: ManagedBarrierStateInner::Stashed { ref mut collected_actors, @@ -206,10 +229,10 @@ impl ManagedBarrierState { }) => { let new = collected_actors.insert(actor_id); assert!(new); - assert_eq!(prev_epoch, barrier.epoch.prev); + assert_eq!(curr_epoch, barrier.epoch.curr); } Some(&mut BarrierState { - prev_epoch, + curr_epoch, inner: ManagedBarrierStateInner::Issued { ref mut remaining_actors, @@ -223,14 +246,14 @@ impl ManagedBarrierState { "the actor doesn't exist. actor_id: {:?}, curr_epoch: {:?}", actor_id, barrier.epoch.curr ); - assert_eq!(prev_epoch, barrier.epoch.prev); - self.may_notify(barrier.epoch.curr); + assert_eq!(curr_epoch, barrier.epoch.curr); + self.may_notify(barrier.epoch.prev); } None => { self.epoch_barrier_state_map.insert( - barrier.epoch.curr, + barrier.epoch.prev, BarrierState { - prev_epoch: barrier.epoch.prev, + curr_epoch: barrier.epoch.curr, inner: ManagedBarrierStateInner::Stashed { collected_actors: once(actor_id).collect(), }, @@ -246,10 +269,14 @@ impl ManagedBarrierState { pub(super) fn transform_to_issued( &mut self, barrier: &Barrier, - actor_ids_to_collect: impl IntoIterator, + actor_ids_to_collect: HashSet, collect_notifier: oneshot::Sender>, - ) -> StreamResult<()> { - let inner = match self.epoch_barrier_state_map.get_mut(&barrier.epoch.curr) { + ) { + let timer = self + .streaming_metrics + .barrier_inflight_latency + .start_timer(); + let inner = match self.epoch_barrier_state_map.get_mut(&barrier.epoch.prev) { Some(&mut BarrierState { inner: ManagedBarrierStateInner::Stashed { @@ -257,18 +284,16 @@ impl ManagedBarrierState { }, .. }) => { - let remaining_actors: HashSet = actor_ids_to_collect - .into_iter() - .filter(|a| !collected_actors.remove(a)) - .collect(); - for (actor_id, err) in &self.failure_actors { - if remaining_actors.contains(actor_id) { - return Err(err.clone()); - } - } - assert!(collected_actors.is_empty()); + assert!( + actor_ids_to_collect.is_superset(collected_actors), + "to_collect: {:?}, collected: {:?}", + actor_ids_to_collect, + collected_actors + ); + let remaining_actors: HashSet = actor_ids_to_collect.sub(collected_actors); ManagedBarrierStateInner::Issued { remaining_actors, + barrier_inflight_latency: timer, collect_notifier: Some(collect_notifier), } } @@ -281,33 +306,21 @@ impl ManagedBarrierState { barrier.epoch ); } - None => { - let remaining_actors: HashSet = actor_ids_to_collect.into_iter().collect(); - // The failure actors could exit before the barrier is issued, while their - // up-downstream actors could be stuck somehow. Return error directly to trigger the - // recovery. - for (actor_id, err) in &self.failure_actors { - if remaining_actors.contains(actor_id) { - return Err(err.clone()); - } - } - ManagedBarrierStateInner::Issued { - remaining_actors, - collect_notifier: Some(collect_notifier), - } - } + None => ManagedBarrierStateInner::Issued { + remaining_actors: actor_ids_to_collect, + barrier_inflight_latency: timer, + collect_notifier: Some(collect_notifier), + }, }; self.epoch_barrier_state_map.insert( - barrier.epoch.curr, + barrier.epoch.prev, BarrierState { - prev_epoch: barrier.epoch.prev, + curr_epoch: barrier.epoch.curr, inner, kind: barrier.kind, }, ); - self.may_notify(barrier.epoch.curr); - - Ok(()) + self.may_notify(barrier.epoch.prev); } } @@ -315,7 +328,6 @@ impl ManagedBarrierState { mod tests { use std::collections::HashSet; - use risingwave_storage::StateStoreImpl; use tokio::sync::oneshot; use crate::executor::Barrier; @@ -323,7 +335,7 @@ mod tests { #[tokio::test] async fn test_managed_state_add_actor() { - let mut managed_barrier_state = ManagedBarrierState::new(StateStoreImpl::for_test()); + let mut managed_barrier_state = ManagedBarrierState::for_test(); let barrier1 = Barrier::new_test_barrier(1); let barrier2 = Barrier::new_test_barrier(2); let barrier3 = Barrier::new_test_barrier(3); @@ -333,15 +345,9 @@ mod tests { let actor_ids_to_collect1 = HashSet::from([1, 2]); let actor_ids_to_collect2 = HashSet::from([1, 2]); let actor_ids_to_collect3 = HashSet::from([1, 2, 3]); - managed_barrier_state - .transform_to_issued(&barrier1, actor_ids_to_collect1, tx1) - .unwrap(); - managed_barrier_state - .transform_to_issued(&barrier2, actor_ids_to_collect2, tx2) - .unwrap(); - managed_barrier_state - .transform_to_issued(&barrier3, actor_ids_to_collect3, tx3) - .unwrap(); + managed_barrier_state.transform_to_issued(&barrier1, actor_ids_to_collect1, tx1); + managed_barrier_state.transform_to_issued(&barrier2, actor_ids_to_collect2, tx2); + managed_barrier_state.transform_to_issued(&barrier3, actor_ids_to_collect3, tx3); managed_barrier_state.collect(1, &barrier1); managed_barrier_state.collect(2, &barrier1); assert_eq!( @@ -350,7 +356,7 @@ mod tests { .first_key_value() .unwrap() .0, - &2 + &1 ); managed_barrier_state.collect(1, &barrier2); managed_barrier_state.collect(1, &barrier3); @@ -361,7 +367,7 @@ mod tests { .first_key_value() .unwrap() .0, - &3 + &2 ); managed_barrier_state.collect(2, &barrier3); managed_barrier_state.collect(3, &barrier3); @@ -370,7 +376,7 @@ mod tests { #[tokio::test] async fn test_managed_state_stop_actor() { - let mut managed_barrier_state = ManagedBarrierState::new(StateStoreImpl::for_test()); + let mut managed_barrier_state = ManagedBarrierState::for_test(); let barrier1 = Barrier::new_test_barrier(1); let barrier2 = Barrier::new_test_barrier(2); let barrier3 = Barrier::new_test_barrier(3); @@ -380,15 +386,9 @@ mod tests { let actor_ids_to_collect1 = HashSet::from([1, 2, 3, 4]); let actor_ids_to_collect2 = HashSet::from([1, 2, 3]); let actor_ids_to_collect3 = HashSet::from([1, 2]); - managed_barrier_state - .transform_to_issued(&barrier1, actor_ids_to_collect1, tx1) - .unwrap(); - managed_barrier_state - .transform_to_issued(&barrier2, actor_ids_to_collect2, tx2) - .unwrap(); - managed_barrier_state - .transform_to_issued(&barrier3, actor_ids_to_collect3, tx3) - .unwrap(); + managed_barrier_state.transform_to_issued(&barrier1, actor_ids_to_collect1, tx1); + managed_barrier_state.transform_to_issued(&barrier2, actor_ids_to_collect2, tx2); + managed_barrier_state.transform_to_issued(&barrier3, actor_ids_to_collect3, tx3); managed_barrier_state.collect(1, &barrier1); managed_barrier_state.collect(1, &barrier2); @@ -402,7 +402,7 @@ mod tests { .first_key_value() .unwrap() .0, - &1 + &0 ); managed_barrier_state.collect(3, &barrier1); managed_barrier_state.collect(3, &barrier2); @@ -412,7 +412,7 @@ mod tests { .first_key_value() .unwrap() .0, - &1 + &0 ); managed_barrier_state.collect(4, &barrier1); assert!(managed_barrier_state.epoch_barrier_state_map.is_empty()); @@ -420,7 +420,7 @@ mod tests { #[tokio::test] async fn test_managed_state_issued_after_collect() { - let mut managed_barrier_state = ManagedBarrierState::new(StateStoreImpl::for_test()); + let mut managed_barrier_state = ManagedBarrierState::for_test(); let barrier1 = Barrier::new_test_barrier(1); let barrier2 = Barrier::new_test_barrier(2); let barrier3 = Barrier::new_test_barrier(3); @@ -438,7 +438,7 @@ mod tests { .first_key_value() .unwrap() .0, - &3 + &2 ); managed_barrier_state.collect(1, &barrier2); assert_eq!( @@ -447,7 +447,7 @@ mod tests { .first_key_value() .unwrap() .0, - &2 + &1 ); managed_barrier_state.collect(1, &barrier1); assert_eq!( @@ -456,25 +456,21 @@ mod tests { .first_key_value() .unwrap() .0, - &1 + &0 ); managed_barrier_state.collect(2, &barrier1); managed_barrier_state.collect(2, &barrier2); managed_barrier_state.collect(2, &barrier3); - managed_barrier_state - .transform_to_issued(&barrier1, actor_ids_to_collect1, tx1) - .unwrap(); + managed_barrier_state.transform_to_issued(&barrier1, actor_ids_to_collect1, tx1); assert_eq!( managed_barrier_state .epoch_barrier_state_map .first_key_value() .unwrap() .0, - &2 + &1 ); - managed_barrier_state - .transform_to_issued(&barrier2, actor_ids_to_collect2, tx2) - .unwrap(); + managed_barrier_state.transform_to_issued(&barrier2, actor_ids_to_collect2, tx2); managed_barrier_state.collect(3, &barrier2); assert_eq!( managed_barrier_state @@ -482,7 +478,7 @@ mod tests { .first_key_value() .unwrap() .0, - &3 + &2 ); managed_barrier_state.collect(3, &barrier3); assert_eq!( @@ -491,11 +487,9 @@ mod tests { .first_key_value() .unwrap() .0, - &3 + &2 ); - managed_barrier_state - .transform_to_issued(&barrier3, actor_ids_to_collect3, tx3) - .unwrap(); + managed_barrier_state.transform_to_issued(&barrier3, actor_ids_to_collect3, tx3); assert!(managed_barrier_state.epoch_barrier_state_map.is_empty()); } } diff --git a/src/stream/src/task/barrier_manager/tests.rs b/src/stream/src/task/barrier_manager/tests.rs index 49e8f0819f7fb..547a4da577d2a 100644 --- a/src/stream/src/task/barrier_manager/tests.rs +++ b/src/stream/src/task/barrier_manager/tests.rs @@ -39,8 +39,10 @@ async fn test_managed_barrier_collection() -> StreamResult<()> { .collect_vec(); // Send a barrier to all actors - let epoch = 114514; - let barrier = Barrier::new_test_barrier(epoch); + let curr_epoch = 114514; + let barrier = Barrier::new_test_barrier(curr_epoch); + let epoch = barrier.epoch.prev; + manager .send_barrier(barrier.clone(), actor_ids.clone(), actor_ids) .await @@ -51,7 +53,7 @@ async fn test_managed_barrier_collection() -> StreamResult<()> { .iter_mut() .map(|(actor_id, rx)| { let barrier = rx.try_recv().unwrap(); - assert_eq!(barrier.epoch.curr, epoch); + assert_eq!(barrier.epoch.prev, epoch); (*actor_id, barrier) }) .collect_vec(); @@ -99,8 +101,9 @@ async fn test_managed_barrier_collection_before_send_request() -> StreamResult<( .collect_vec(); // Prepare the barrier - let epoch = 114514; - let barrier = Barrier::new_test_barrier(epoch); + let curr_epoch = 114514; + let barrier = Barrier::new_test_barrier(curr_epoch); + let epoch = barrier.epoch.prev; // Collect a barrier before sending manager.collect(extra_actor_id, &barrier); @@ -117,7 +120,7 @@ async fn test_managed_barrier_collection_before_send_request() -> StreamResult<( .iter_mut() .map(|(actor_id, rx)| { let barrier = rx.try_recv().unwrap(); - assert_eq!(barrier.epoch.curr, epoch); + assert_eq!(barrier.epoch.prev, epoch); (*actor_id, barrier) }) .collect_vec(); diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 57c5f12612aae..db60f43ea95dd 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -270,7 +270,6 @@ impl LocalStreamManager { .expect("no rx for local mode") .await .context("failed to collect barrier")??; - complete_receiver.barrier_inflight_timer.observe_duration(); Ok(result) } From a756cbb624f645bca48dd76c59bae877039adde6 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 10 Jan 2024 14:40:21 +0800 Subject: [PATCH 11/17] feat(frontend): return `NotSupportedError` for `EXPLAIN CREATE VIEW` (#14475) --- e2e_test/batch/explain.slt | 3 +++ src/frontend/src/handler/explain.rs | 10 +++++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/e2e_test/batch/explain.slt b/e2e_test/batch/explain.slt index cff6af31f2470..c3f4bfaf32f0e 100644 --- a/e2e_test/batch/explain.slt +++ b/e2e_test/batch/explain.slt @@ -9,3 +9,6 @@ explain create sink sink_t from t with ( connector = 'kafka', type = 'append-onl statement ok drop table t; + +statement error Not supported: EXPLAIN CREATE VIEW +explain create view v as select 1; \ No newline at end of file diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index c93f4996fac23..826934be997fd 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -111,7 +111,15 @@ async fn do_handle_explain( emit_mode, ) .map(|x| x.0), - + Statement::CreateView { + materialized: false, + .. + } => { + return Err(ErrorCode::NotSupported( + "EXPLAIN CREATE VIEW".into(), + "A created VIEW is just an alias. Instead, use EXPLAIN on the queries which reference the view.".into() + ).into()); + } Statement::CreateSink { stmt } => { gen_sink_plan(&session, context.clone(), stmt).map(|plan| plan.sink_plan) } From df1b9de16492887831a31028314da8fff848ff3a Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 10 Jan 2024 15:03:55 +0800 Subject: [PATCH 12/17] feat(common): expose `OrderType::direction()` (#14447) Signed-off-by: Richard Chien --- src/common/src/util/sort_util.rs | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/common/src/util/sort_util.rs b/src/common/src/util/sort_util.rs index 6cfca150a4dcc..9c2a8c05b6282 100644 --- a/src/common/src/util/sort_util.rs +++ b/src/common/src/util/sort_util.rs @@ -29,7 +29,7 @@ use crate::types::{DefaultOrdered, ToDatumRef}; /// Sort direction, ascending/descending. #[derive(PartialEq, Eq, Hash, Copy, Clone, Debug, Display, Default)] -enum Direction { +pub enum Direction { #[default] #[display("ASC")] Ascending, @@ -38,7 +38,7 @@ enum Direction { } impl Direction { - pub fn from_protobuf(direction: &PbDirection) -> Self { + fn from_protobuf(direction: &PbDirection) -> Self { match direction { PbDirection::Ascending => Self::Ascending, PbDirection::Descending => Self::Descending, @@ -46,7 +46,7 @@ impl Direction { } } - pub fn to_protobuf(self) -> PbDirection { + fn to_protobuf(self) -> PbDirection { match self { Self::Ascending => PbDirection::Ascending, Self::Descending => PbDirection::Descending, @@ -74,7 +74,7 @@ enum NullsAre { } impl NullsAre { - pub fn from_protobuf(nulls_are: &PbNullsAre) -> Self { + fn from_protobuf(nulls_are: &PbNullsAre) -> Self { match nulls_are { PbNullsAre::Largest => Self::Largest, PbNullsAre::Smallest => Self::Smallest, @@ -82,7 +82,7 @@ impl NullsAre { } } - pub fn to_protobuf(self) -> PbNullsAre { + fn to_protobuf(self) -> PbNullsAre { match self { Self::Largest => PbNullsAre::Largest, Self::Smallest => PbNullsAre::Smallest, @@ -185,6 +185,10 @@ impl OrderType { Self::nulls_last(Direction::Descending) } + pub fn direction(&self) -> Direction { + self.direction + } + pub fn is_ascending(&self) -> bool { self.direction == Direction::Ascending } From f10bbd5ff1d9430fcee8929261154a79374d08d5 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 10 Jan 2024 16:04:46 +0800 Subject: [PATCH 13/17] refactor(over window): extract `range_utils` which handles `Range` operations (#14474) Signed-off-by: Richard Chien --- .../core/src/window_function/state/buffer.rs | 139 +------------- .../core/src/window_function/state/mod.rs | 4 +- .../src/window_function/state/range_utils.rs | 177 ++++++++++++++++++ 3 files changed, 181 insertions(+), 139 deletions(-) create mode 100644 src/expr/core/src/window_function/state/range_utils.rs diff --git a/src/expr/core/src/window_function/state/buffer.rs b/src/expr/core/src/window_function/state/buffer.rs index 0a3e2fb27ab7c..54227084f20f5 100644 --- a/src/expr/core/src/window_function/state/buffer.rs +++ b/src/expr/core/src/window_function/state/buffer.rs @@ -16,8 +16,9 @@ use std::collections::VecDeque; use std::ops::Range; use risingwave_common::array::Op; -use smallvec::{smallvec, SmallVec}; +use super::range_utils::range_except; +use crate::window_function::state::range_utils::range_diff; use crate::window_function::{Frame, FrameBounds, FrameExclusion}; struct Entry { @@ -260,149 +261,13 @@ impl WindowBuffer { } } -/// Calculate range (A - B), the result might be the union of two ranges when B is totally included -/// in the A. -fn range_except(a: Range, b: Range) -> (Range, Range) { - #[allow(clippy::if_same_then_else)] // for better readability - if a.is_empty() { - (0..0, 0..0) - } else if b.is_empty() { - (a, 0..0) - } else if a.end <= b.start || b.end <= a.start { - // a: [ ) - // b: [ ) - // or - // a: [ ) - // b: [ ) - (a, 0..0) - } else if b.start <= a.start && a.end <= b.end { - // a: [ ) - // b: [ ) - (0..0, 0..0) - } else if a.start < b.start && b.end < a.end { - // a: [ ) - // b: [ ) - (a.start..b.start, b.end..a.end) - } else if a.end <= b.end { - // a: [ ) - // b: [ ) - (a.start..b.start, 0..0) - } else if b.start <= a.start { - // a: [ ) - // b: [ ) - (b.end..a.end, 0..0) - } else { - unreachable!() - } -} - -/// Calculate the difference of two ranges A and B, return (removed ranges, added ranges). -/// Note this is quite different from [`range_except`]. -#[allow(clippy::type_complexity)] // looks complex but it's not -fn range_diff( - a: Range, - b: Range, -) -> (SmallVec<[Range; 2]>, SmallVec<[Range; 2]>) { - if a.start == b.start { - match a.end.cmp(&b.end) { - std::cmp::Ordering::Equal => { - // a: [ ) - // b: [ ) - (smallvec![], smallvec![]) - } - std::cmp::Ordering::Less => { - // a: [ ) - // b: [ ) - (smallvec![], smallvec![a.end..b.end]) - } - std::cmp::Ordering::Greater => { - // a: [ ) - // b: [ ) - (smallvec![b.end..a.end], smallvec![]) - } - } - } else if a.end == b.end { - debug_assert!(a.start != b.start); - if a.start < b.start { - // a: [ ) - // b: [ ) - (smallvec![a.start..b.start], smallvec![]) - } else { - // a: [ ) - // b: [ ) - (smallvec![], smallvec![b.start..a.start]) - } - } else { - debug_assert!(a.start != b.start && a.end != b.end); - if a.end <= b.start || b.end <= a.start { - // a: [ ) - // b: [ [ ) - // or - // a: [ ) - // b: [ ) ) - (smallvec![a], smallvec![b]) - } else if b.start < a.start && a.end < b.end { - // a: [ ) - // b: [ ) - (smallvec![], smallvec![b.start..a.start, a.end..b.end]) - } else if a.start < b.start && b.end < a.end { - // a: [ ) - // b: [ ) - (smallvec![a.start..b.start, b.end..a.end], smallvec![]) - } else if a.end < b.end { - // a: [ ) - // b: [ ) - (smallvec![a.start..b.start], smallvec![a.end..b.end]) - } else { - // a: [ ) - // b: [ ) - (smallvec![b.end..a.end], smallvec![b.start..a.start]) - } - } -} - #[cfg(test)] mod tests { - use std::collections::HashSet; - use itertools::Itertools; use super::*; use crate::window_function::{Frame, FrameBound}; - #[test] - fn test_range_diff() { - fn test( - a: Range, - b: Range, - expected_removed: impl IntoIterator, - expected_added: impl IntoIterator, - ) { - let (removed, added) = range_diff(a, b); - let removed_set = removed.into_iter().flatten().collect::>(); - let added_set = added.into_iter().flatten().collect::>(); - let expected_removed_set = expected_removed.into_iter().collect::>(); - let expected_added_set = expected_added.into_iter().collect::>(); - assert_eq!(removed_set, expected_removed_set); - assert_eq!(added_set, expected_added_set); - } - - test(0..0, 0..0, [], []); - test(0..1, 0..1, [], []); - test(0..1, 0..2, [], [1]); - test(0..2, 0..1, [1], []); - test(0..2, 1..2, [0], []); - test(1..2, 0..2, [], [0]); - test(0..1, 1..2, [0], [1]); - test(0..1, 2..3, [0], [2]); - test(1..2, 0..1, [1], [0]); - test(2..3, 0..1, [2], [0]); - test(0..3, 1..2, [0, 2], []); - test(1..2, 0..3, [], [0, 2]); - test(0..3, 2..4, [0, 1], [3]); - test(2..4, 0..3, [3], [0, 1]); - } - #[test] fn test_rows_frame_unbounded_preceding_to_current_row() { let mut buffer = WindowBuffer::new( diff --git a/src/expr/core/src/window_function/state/mod.rs b/src/expr/core/src/window_function/state/mod.rs index 805688f22b3b1..37ee086ca7ba4 100644 --- a/src/expr/core/src/window_function/state/mod.rs +++ b/src/expr/core/src/window_function/state/mod.rs @@ -24,9 +24,9 @@ use smallvec::SmallVec; use super::{WindowFuncCall, WindowFuncKind}; use crate::{ExprError, Result}; -mod buffer; - mod aggregate; +mod buffer; +mod range_utils; mod rank; /// Unique and ordered identifier for a row in internal states. diff --git a/src/expr/core/src/window_function/state/range_utils.rs b/src/expr/core/src/window_function/state/range_utils.rs new file mode 100644 index 0000000000000..d0dd07f223041 --- /dev/null +++ b/src/expr/core/src/window_function/state/range_utils.rs @@ -0,0 +1,177 @@ +// 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 std::ops::Range; + +use smallvec::{smallvec, SmallVec}; + +/// Calculate range (A - B), the result might be the union of two ranges when B is totally included +/// in the A. +pub(super) fn range_except(a: Range, b: Range) -> (Range, Range) { + #[allow(clippy::if_same_then_else)] // for better readability + if a.is_empty() { + (0..0, 0..0) + } else if b.is_empty() { + (a, 0..0) + } else if a.end <= b.start || b.end <= a.start { + // a: [ ) + // b: [ ) + // or + // a: [ ) + // b: [ ) + (a, 0..0) + } else if b.start <= a.start && a.end <= b.end { + // a: [ ) + // b: [ ) + (0..0, 0..0) + } else if a.start < b.start && b.end < a.end { + // a: [ ) + // b: [ ) + (a.start..b.start, b.end..a.end) + } else if a.end <= b.end { + // a: [ ) + // b: [ ) + (a.start..b.start, 0..0) + } else if b.start <= a.start { + // a: [ ) + // b: [ ) + (b.end..a.end, 0..0) + } else { + unreachable!() + } +} + +/// Calculate the difference of two ranges A and B, return (removed ranges, added ranges). +/// Note this is quite different from [`range_except`]. +#[allow(clippy::type_complexity)] // looks complex but it's not +pub(super) fn range_diff( + a: Range, + b: Range, +) -> (SmallVec<[Range; 2]>, SmallVec<[Range; 2]>) { + if a.start == b.start { + match a.end.cmp(&b.end) { + std::cmp::Ordering::Equal => { + // a: [ ) + // b: [ ) + (smallvec![], smallvec![]) + } + std::cmp::Ordering::Less => { + // a: [ ) + // b: [ ) + (smallvec![], smallvec![a.end..b.end]) + } + std::cmp::Ordering::Greater => { + // a: [ ) + // b: [ ) + (smallvec![b.end..a.end], smallvec![]) + } + } + } else if a.end == b.end { + debug_assert!(a.start != b.start); + if a.start < b.start { + // a: [ ) + // b: [ ) + (smallvec![a.start..b.start], smallvec![]) + } else { + // a: [ ) + // b: [ ) + (smallvec![], smallvec![b.start..a.start]) + } + } else { + debug_assert!(a.start != b.start && a.end != b.end); + if a.end <= b.start || b.end <= a.start { + // a: [ ) + // b: [ [ ) + // or + // a: [ ) + // b: [ ) ) + (smallvec![a], smallvec![b]) + } else if b.start < a.start && a.end < b.end { + // a: [ ) + // b: [ ) + (smallvec![], smallvec![b.start..a.start, a.end..b.end]) + } else if a.start < b.start && b.end < a.end { + // a: [ ) + // b: [ ) + (smallvec![a.start..b.start, b.end..a.end], smallvec![]) + } else if a.end < b.end { + // a: [ ) + // b: [ ) + (smallvec![a.start..b.start], smallvec![a.end..b.end]) + } else { + // a: [ ) + // b: [ ) + (smallvec![b.end..a.end], smallvec![b.start..a.start]) + } + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + + use super::*; + + #[test] + fn test_range_except() { + fn test(a: Range, b: Range, expected: impl IntoIterator) { + let (l, r) = range_except(a, b); + let set = l.into_iter().chain(r).collect::>(); + assert_eq!(set, expected.into_iter().collect()) + } + + test(0..0, 0..0, []); + test(0..1, 0..1, []); + test(0..1, 0..2, []); + test(1..2, 0..2, []); + test(0..2, 0..1, [1]); + test(0..2, 1..2, [0]); + test(0..5, 2..3, [0, 1, 3, 4]); + test(2..5, 1..3, [3, 4]); + test(2..5, 4..5, [2, 3]); + } + + #[test] + fn test_range_diff() { + fn test( + a: Range, + b: Range, + expected_removed: impl IntoIterator, + expected_added: impl IntoIterator, + ) { + let (removed, added) = range_diff(a, b); + let removed_set = removed.into_iter().flatten().collect::>(); + let added_set = added.into_iter().flatten().collect::>(); + let expected_removed_set = expected_removed.into_iter().collect::>(); + let expected_added_set = expected_added.into_iter().collect::>(); + assert_eq!(removed_set, expected_removed_set); + assert_eq!(added_set, expected_added_set); + } + + test(0..0, 0..0, [], []); + test(0..1, 0..1, [], []); + test(0..1, 0..2, [], [1]); + test(0..2, 0..1, [1], []); + test(0..2, 1..2, [0], []); + test(1..2, 0..2, [], [0]); + test(0..1, 1..2, [0], [1]); + test(0..1, 2..3, [0], [2]); + test(1..2, 0..1, [1], [0]); + test(2..3, 0..1, [2], [0]); + test(0..3, 1..2, [0, 2], []); + test(1..2, 0..3, [], [0, 2]); + test(0..3, 2..4, [0, 1], [3]); + test(2..4, 0..3, [3], [0, 1]); + } +} From 6c5b56d1fb929e2edf87ba15f00bc7c38b4d2508 Mon Sep 17 00:00:00 2001 From: Wallace Date: Wed, 10 Jan 2024 16:40:23 +0800 Subject: [PATCH 14/17] fix(meta): do not split by vnode for low write throughput (#12534) Signed-off-by: Little-Wallace --- proto/hummock.proto | 5 +- src/common/src/config.rs | 2 +- src/config/example.toml | 2 +- src/meta/src/hummock/compaction/mod.rs | 50 +------ .../picker/base_level_compaction_picker.rs | 31 ++++- .../picker/compaction_task_validator.rs | 1 - .../picker/intra_compaction_picker.rs | 131 ++++++++++++++++-- .../picker/manual_compaction_picker.rs | 29 ++-- .../picker/min_overlap_compaction_picker.rs | 51 +++---- src/meta/src/hummock/compaction/picker/mod.rs | 16 +++ .../picker/space_reclaim_compaction_picker.rs | 4 +- .../picker/tier_compaction_picker.rs | 9 +- .../picker/ttl_reclaim_compaction_picker.rs | 1 + .../compaction/selector/level_selector.rs | 81 ++++++----- .../src/hummock/compaction/selector/mod.rs | 6 + src/meta/src/hummock/manager/mod.rs | 77 +++++++--- src/meta/src/hummock/metrics_utils.rs | 31 ++++- .../compaction_group/hummock_version_ext.rs | 30 ++++ .../hummock_test/src/compactor_tests.rs | 2 +- 19 files changed, 377 insertions(+), 182 deletions(-) diff --git a/proto/hummock.proto b/proto/hummock.proto index 58007117811d5..ae197caeab29e 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -49,6 +49,7 @@ message Level { uint64 total_file_size = 4; uint64 sub_level_id = 5; uint64 uncompressed_file_size = 6; + uint32 vnode_partition_count = 7; } message InputLevel { @@ -62,6 +63,7 @@ message IntraLevelDelta { uint64 l0_sub_level_id = 2; repeated uint64 removed_table_ids = 3; repeated SstableInfo inserted_table_infos = 4; + uint32 vnode_partition_count = 5; } enum CompatibilityVersion { @@ -142,6 +144,7 @@ message HummockVersion { uint64 group_id = 3; uint64 parent_group_id = 4; repeated uint32 member_table_ids = 5; + uint32 vnode_partition_count = 6; } uint64 id = 1; // Levels of each compaction group @@ -342,7 +345,7 @@ message CompactTask { bool split_by_state_table = 21 [deprecated = true]; // Compaction needs to cut the state table every time 1/weight of vnodes in the table have been processed. // Deprecated. use table_vnode_partition instead; - uint32 split_weight_by_vnode = 22 [deprecated = true]; + uint32 split_weight_by_vnode = 22; map table_vnode_partition = 23; // The table watermark of any table id. In compaction we only use the table watermarks on safe epoch, // so we only need to include the table watermarks on safe epoch to reduce the size of metadata. diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 67a3e5e49b858..3c7e199a649e9 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -1034,7 +1034,7 @@ pub mod default { } pub fn partition_vnode_count() -> u32 { - 64 + 16 } pub fn table_write_throughput_threshold() -> u64 { diff --git a/src/config/example.toml b/src/config/example.toml index de22d4ef87a37..1a94eaa969073 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -40,7 +40,7 @@ move_table_size_limit = 10737418240 split_group_size_limit = 68719476736 cut_table_size_limit = 1073741824 do_not_config_object_storage_lifecycle = false -partition_vnode_count = 64 +partition_vnode_count = 16 table_write_throughput_threshold = 16777216 min_table_split_write_throughput = 4194304 compaction_task_max_heartbeat_interval_secs = 60 diff --git a/src/meta/src/hummock/compaction/mod.rs b/src/meta/src/hummock/compaction/mod.rs index 4cd913fa7d7fa..562027b91c8da 100644 --- a/src/meta/src/hummock/compaction/mod.rs +++ b/src/meta/src/hummock/compaction/mod.rs @@ -17,23 +17,20 @@ pub mod compaction_config; mod overlap_strategy; use risingwave_common::catalog::TableOption; -use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; -use risingwave_pb::hummock::compact_task::{self, TaskStatus, TaskType}; +use risingwave_pb::hummock::compact_task::{self, TaskType}; mod picker; pub mod selector; -use std::collections::{BTreeMap, HashMap, HashSet}; +use std::collections::{HashMap, HashSet}; use std::fmt::{Debug, Formatter}; use std::sync::Arc; use picker::{LevelCompactionPicker, TierCompactionPicker}; -use risingwave_hummock_sdk::{ - can_concat, CompactionGroupId, HummockCompactionTaskId, HummockEpoch, -}; +use risingwave_hummock_sdk::{can_concat, CompactionGroupId, HummockCompactionTaskId}; use risingwave_pb::hummock::compaction_config::CompactionMode; use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{CompactTask, CompactionConfig, KeyRange, LevelType}; +use risingwave_pb::hummock::{CompactTask, CompactionConfig, LevelType}; pub use selector::CompactionSelector; use self::selector::LocalSelectorStatistic; @@ -107,51 +104,18 @@ impl CompactStatus { stats: &mut LocalSelectorStatistic, selector: &mut Box, table_id_to_options: HashMap, - ) -> Option { + ) -> Option { // When we compact the files, we must make the result of compaction meet the following // conditions, for any user key, the epoch of it in the file existing in the lower // layer must be larger. - let ret = selector.pick_compaction( + selector.pick_compaction( task_id, group, levels, &mut self.level_handlers, stats, table_id_to_options, - )?; - let target_level_id = ret.input.target_level; - - let compression_algorithm = match ret.compression_algorithm.as_str() { - "Lz4" => 1, - "Zstd" => 2, - _ => 0, - }; - - let compact_task = CompactTask { - input_ssts: ret.input.input_levels, - splits: vec![KeyRange::inf()], - watermark: HummockEpoch::MAX, - sorted_output_ssts: vec![], - task_id, - target_level: target_level_id as u32, - // only gc delete keys in last level because there may be older version in more bottom - // level. - gc_delete_keys: target_level_id == self.level_handlers.len() - 1, - base_level: ret.base_level as u32, - task_status: TaskStatus::Pending as i32, - compaction_group_id: group.group_id, - existing_table_ids: vec![], - compression_algorithm, - target_file_size: ret.target_file_size, - compaction_filter_mask: 0, - table_options: BTreeMap::default(), - current_epoch_time: 0, - target_sub_level_id: ret.input.target_sub_level_id, - task_type: ret.compaction_task_type as i32, - table_vnode_partition: BTreeMap::default(), - ..Default::default() - }; - Some(compact_task) + ) } pub fn is_trivial_move_task(task: &CompactTask) -> bool { diff --git a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs index 2005d7ec42948..c5c09f1544ce7 100644 --- a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs @@ -60,12 +60,13 @@ impl CompactionPicker for LevelCompactionPicker { return None; } - if let Some(ret) = self.pick_base_trivial_move( + if let Some(mut ret) = self.pick_base_trivial_move( l0, levels.get_level(self.target_level), level_handlers, stats, ) { + ret.vnode_partition_count = levels.vnode_partition_count; return Some(ret); } @@ -73,6 +74,7 @@ impl CompactionPicker for LevelCompactionPicker { if let Some(ret) = self.pick_multi_level_to_base( l0, levels.get_level(self.target_level), + levels.vnode_partition_count, level_handlers, stats, ) { @@ -128,6 +130,7 @@ impl LevelCompactionPicker { &self, l0: &OverlappingLevel, target_level: &Level, + vnode_partition_count: u32, level_handlers: &[LevelHandler], stats: &mut LocalPickerStatistic, ) -> Option { @@ -147,8 +150,18 @@ impl LevelCompactionPicker { overlap_strategy.clone(), ); - let l0_select_tables_vec = non_overlap_sub_level_picker - .pick_l0_multi_non_overlap_level(&l0.sub_levels, &level_handlers[0]); + let mut max_vnode_partition_idx = 0; + for (idx, level) in l0.sub_levels.iter().enumerate() { + if level.vnode_partition_count < vnode_partition_count { + break; + } + max_vnode_partition_idx = idx; + } + + let l0_select_tables_vec = non_overlap_sub_level_picker.pick_l0_multi_non_overlap_level( + &l0.sub_levels[..=max_vnode_partition_idx], + &level_handlers[0], + ); if l0_select_tables_vec.is_empty() { stats.skip_by_pending_files += 1; return None; @@ -217,6 +230,7 @@ impl LevelCompactionPicker { select_input_size: input.total_file_size, target_input_size: target_file_size, total_file_count: (input.total_file_count + target_file_count) as u64, + vnode_partition_count, ..Default::default() }; @@ -225,6 +239,15 @@ impl LevelCompactionPicker { ValidationRuleType::ToBase, stats, ) { + if l0.total_file_size > target_level.total_file_size * 8 { + tracing::warn!("skip task with level count: {}, file count: {}, select size: {}, target size: {}, target level size: {}", + result.input_levels.len(), + result.total_file_count, + result.select_input_size, + result.target_input_size, + target_level.total_file_size, + ); + } continue; } @@ -423,6 +446,7 @@ pub mod tests { total_file_size: 0, sub_level_id: 0, uncompressed_file_size: 0, + ..Default::default() }]; let mut levels = Levels { levels, @@ -487,6 +511,7 @@ pub mod tests { total_file_size: 900, sub_level_id: 0, uncompressed_file_size: 900, + ..Default::default() }], l0: Some(generate_l0_nonoverlapping_sublevels(vec![])), ..Default::default() diff --git a/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs b/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs index dee2dd8e71aa2..860d211239bd5 100644 --- a/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs +++ b/src/meta/src/hummock/compaction/picker/compaction_task_validator.rs @@ -118,7 +118,6 @@ impl CompactionTaskValidationRule for TierCompactionTaskValidationRule { stats.skip_by_count_limit += 1; return false; } - true } } diff --git a/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs index 5e3878b2fa9d0..034db2b127963 100644 --- a/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs @@ -57,7 +57,16 @@ impl CompactionPicker for IntraCompactionPicker { return None; } - if let Some(ret) = self.pick_l0_intra(l0, &level_handlers[0], stats) { + let vnode_partition_count = levels.vnode_partition_count; + + if let Some(ret) = + self.pick_whole_level(l0, &level_handlers[0], vnode_partition_count, stats) + { + return Some(ret); + } + + if let Some(ret) = self.pick_l0_intra(l0, &level_handlers[0], vnode_partition_count, stats) + { return Some(ret); } @@ -84,13 +93,102 @@ impl IntraCompactionPicker { } } + fn pick_whole_level( + &self, + l0: &OverlappingLevel, + level_handler: &LevelHandler, + partition_count: u32, + stats: &mut LocalPickerStatistic, + ) -> Option { + if partition_count == 0 { + return None; + } + for (idx, level) in l0.sub_levels.iter().enumerate() { + if level.level_type() != LevelType::Nonoverlapping + || level.vnode_partition_count == partition_count + { + continue; + } + + let max_compaction_bytes = std::cmp::max( + self.config.max_bytes_for_level_base, + self.config.sub_level_max_compaction_bytes + * (self.config.level0_sub_level_compact_level_count as u64), + ); + + let mut select_input_size = 0; + + let mut select_level_inputs = vec![]; + let mut total_file_count = 0; + let mut wait_enough = false; + for next_level in l0.sub_levels.iter().skip(idx) { + if select_input_size > max_compaction_bytes + || total_file_count > self.config.level0_max_compact_file_number + || (next_level.vnode_partition_count == partition_count + && select_level_inputs.len() > 1) + { + wait_enough = true; + break; + } + + if level_handler.is_level_pending_compact(next_level) { + break; + } + + select_input_size += next_level.total_file_size; + total_file_count += next_level.table_infos.len() as u64; + + select_level_inputs.push(InputLevel { + level_idx: 0, + level_type: next_level.level_type, + table_infos: next_level.table_infos.clone(), + }); + } + if !select_level_inputs.is_empty() { + let vnode_partition_count = + if select_input_size > self.config.sub_level_max_compaction_bytes / 2 { + partition_count + } else { + 0 + }; + let result = CompactionInput { + input_levels: select_level_inputs, + target_sub_level_id: level.sub_level_id, + select_input_size, + total_file_count, + vnode_partition_count, + ..Default::default() + }; + if wait_enough + || self.compaction_task_validator.valid_compact_task( + &result, + ValidationRuleType::Intra, + stats, + ) + { + return Some(result); + } + } + } + + None + } + fn pick_l0_intra( &self, l0: &OverlappingLevel, level_handler: &LevelHandler, + vnode_partition_count: u32, stats: &mut LocalPickerStatistic, ) -> Option { let overlap_strategy = create_overlap_strategy(self.config.compaction_mode()); + let mut max_vnode_partition_idx = 0; + for (idx, level) in l0.sub_levels.iter().enumerate() { + if level.vnode_partition_count < vnode_partition_count { + break; + } + max_vnode_partition_idx = idx; + } for (idx, level) in l0.sub_levels.iter().enumerate() { if level.level_type() != LevelType::Nonoverlapping @@ -99,6 +197,10 @@ impl IntraCompactionPicker { continue; } + if idx > max_vnode_partition_idx { + break; + } + if level_handler.is_level_all_pending_compact(level) { continue; } @@ -117,7 +219,10 @@ impl IntraCompactionPicker { ); let l0_select_tables_vec = non_overlap_sub_level_picker - .pick_l0_multi_non_overlap_level(&l0.sub_levels[idx..], level_handler); + .pick_l0_multi_non_overlap_level( + &l0.sub_levels[idx..=max_vnode_partition_idx], + level_handler, + ); if l0_select_tables_vec.is_empty() { continue; @@ -192,6 +297,12 @@ impl IntraCompactionPicker { continue; } + if l0.sub_levels[idx + 1].vnode_partition_count + != l0.sub_levels[idx].vnode_partition_count + { + continue; + } + let trivial_move_picker = TrivialMovePicker::new(0, 0, overlap_strategy.clone()); let select_sst = trivial_move_picker.pick_trivial_move_sst( @@ -281,14 +392,11 @@ pub mod tests { fn test_l0_to_l1_compact_conflict() { // When picking L0->L1, L0's selecting_key_range should not be overlapped with L0's // compacting_key_range. - let mut picker = create_compaction_picker_for_test(); let levels = vec![Level { level_idx: 1, level_type: LevelType::Nonoverlapping as i32, table_infos: vec![], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }]; let mut levels = Levels { levels, @@ -307,14 +415,9 @@ pub mod tests { generate_table(2, 1, 350, 500, 2), ], ); - let mut levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; + let levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; let mut local_stats = LocalPickerStatistic::default(); - let ret = picker - .pick_compaction(&levels, &levels_handler, &mut local_stats) - .unwrap(); - // trivial_move - ret.add_pending_task(0, &mut levels_handler); // pending only for test push_tables_level0_nonoverlapping(&mut levels, vec![generate_table(3, 1, 250, 300, 3)]); let config: CompactionConfig = CompactionConfigBuilder::new() .level0_tier_compact_file_number(2) @@ -341,9 +444,7 @@ pub mod tests { level_idx: 1, level_type: LevelType::Nonoverlapping as i32, table_infos: vec![generate_table(3, 1, 200, 300, 2)], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }], l0: Some(generate_l0_nonoverlapping_sublevels(vec![ generate_table(1, 1, 100, 210, 2), diff --git a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs index b0d1934823512..b5f187a3253d0 100644 --- a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs @@ -384,9 +384,7 @@ pub mod tests { generate_table(1, 1, 101, 200, 1), generate_table(2, 1, 222, 300, 1), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 2, @@ -398,9 +396,7 @@ pub mod tests { generate_table(7, 1, 501, 800, 1), generate_table(8, 2, 301, 400, 1), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, ]; let mut levels = Levels { @@ -562,9 +558,7 @@ pub mod tests { generate_table(3, 1, 0, 100, 1), generate_table(4, 2, 2000, 3000, 1), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 2, @@ -573,9 +567,7 @@ pub mod tests { generate_table(1, 1, 0, 100, 1), generate_table(2, 2, 2000, 3000, 1), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, ]; // Set internal_table_ids. @@ -617,9 +609,7 @@ pub mod tests { generate_table(3, 2, 200, 300, 1), generate_table(4, 2, 300, 400, 1), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }]; let levels = Levels { levels, @@ -641,6 +631,7 @@ pub mod tests { total_file_size: 0, sub_level_id: 0, uncompressed_file_size: 0, + ..Default::default() }]; let levels = Levels { levels, @@ -1177,9 +1168,7 @@ pub mod tests { generate_table(3, 1, 101, 200, 1), generate_table(4, 1, 222, 300, 1), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, ]; assert_eq!(levels.len(), 4); @@ -1287,9 +1276,7 @@ pub mod tests { generate_table(6, 1, 444, 500, 1), generate_table(7, 1, 555, 600, 1), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, ]; assert_eq!(levels.len(), 4); diff --git a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs index ff9a6cda8dc3b..3065d108e700f 100644 --- a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs @@ -163,6 +163,7 @@ impl CompactionPicker for MinOverlappingPicker { }, ], target_level: self.target_level, + vnode_partition_count: levels.vnode_partition_count, ..Default::default() }) } @@ -434,10 +435,7 @@ pub mod tests { generate_table(1, 1, 101, 200, 1), generate_table(2, 1, 222, 300, 1), ], - - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 2, @@ -449,9 +447,7 @@ pub mod tests { generate_table(7, 1, 501, 800, 1), generate_table(8, 2, 301, 400, 1), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, ]; let levels = Levels { @@ -511,9 +507,7 @@ pub mod tests { generate_table(1, 1, 100, 149, 2), generate_table(2, 1, 150, 249, 2), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 2, @@ -522,9 +516,7 @@ pub mod tests { generate_table(4, 1, 50, 199, 1), generate_table(5, 1, 200, 399, 1), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, ]; let levels = Levels { @@ -573,8 +565,7 @@ pub mod tests { generate_table(8, 1, 450, 500, 2), ], total_file_size: 800, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 2, @@ -587,8 +578,7 @@ pub mod tests { generate_table(11, 1, 450, 500, 2), ], total_file_size: 250, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 3, @@ -599,8 +589,7 @@ pub mod tests { generate_table(13, 1, 450, 500, 2), ], total_file_size: 150, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 4, @@ -611,8 +600,7 @@ pub mod tests { generate_table(16, 1, 450, 500, 2), ], total_file_size: 150, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, ]; @@ -677,8 +665,7 @@ pub mod tests { generate_table(8, 1, 450, 500, 2), ], total_file_size: 800, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 2, @@ -691,8 +678,7 @@ pub mod tests { generate_table(11, 1, 450, 500, 2), ], total_file_size: 250, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 3, @@ -703,8 +689,7 @@ pub mod tests { generate_table(13, 1, 450, 500, 2), ], total_file_size: 150, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 4, @@ -715,8 +700,7 @@ pub mod tests { generate_table(16, 1, 450, 500, 2), ], total_file_size: 150, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, ]; @@ -806,8 +790,7 @@ pub mod tests { level_type: LevelType::Nonoverlapping as i32, table_infos: vec![generate_table(0, 1, 400, 500, 2)], total_file_size: 100, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 2, @@ -817,8 +800,7 @@ pub mod tests { generate_table(2, 1, 600, 700, 1), ], total_file_size: 200, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, Level { level_idx: 3, @@ -828,8 +810,7 @@ pub mod tests { generate_table(4, 1, 600, 800, 1), ], total_file_size: 400, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, ]; diff --git a/src/meta/src/hummock/compaction/picker/mod.rs b/src/meta/src/hummock/compaction/picker/mod.rs index 3c6b066e7f5e2..7e33123272684 100644 --- a/src/meta/src/hummock/compaction/picker/mod.rs +++ b/src/meta/src/hummock/compaction/picker/mod.rs @@ -61,6 +61,7 @@ pub struct CompactionInput { pub select_input_size: u64, pub target_input_size: u64, pub total_file_count: u64, + pub vnode_partition_count: u32, } impl CompactionInput { @@ -96,3 +97,18 @@ pub trait CompactionPicker { stats: &mut LocalPickerStatistic, ) -> Option; } + +#[derive(Default, Clone, Debug)] +pub struct PartitionLevelInfo { + pub level_id: u32, + pub sub_level_id: u64, + pub left_idx: usize, + pub right_idx: usize, + pub total_file_size: u64, +} + +#[derive(Default, Clone, Debug)] +pub struct LevelPartition { + pub sub_levels: Vec, + pub total_file_size: u64, +} diff --git a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs index 928b4d4d454c3..e14baaf086c1f 100644 --- a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs @@ -221,9 +221,7 @@ mod test { generate_table_with_ids_and_epochs(10, 1, 888, 1600, 1, vec![10], 0, 0), generate_table_with_ids_and_epochs(11, 1, 1600, 1800, 1, vec![10], 0, 0), ], - total_file_size: 0, - sub_level_id: 0, - uncompressed_file_size: 0, + ..Default::default() }, ]; diff --git a/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs index b2a058659fea4..20b7d3b38c8e7 100644 --- a/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/tier_compaction_picker.rs @@ -52,6 +52,7 @@ impl TierCompactionPicker { &self, l0: &OverlappingLevel, level_handler: &LevelHandler, + mut vnode_partition_count: u32, stats: &mut LocalPickerStatistic, ) -> Option { for (idx, level) in l0.sub_levels.iter().enumerate() { @@ -114,6 +115,9 @@ impl TierCompactionPicker { } select_level_inputs.reverse(); + if compaction_bytes < self.config.sub_level_max_compaction_bytes / 2 { + vnode_partition_count = 0; + } let result = CompactionInput { input_levels: select_level_inputs, @@ -122,6 +126,7 @@ impl TierCompactionPicker { select_input_size: compaction_bytes, target_input_size: 0, total_file_count: compact_file_count, + vnode_partition_count, }; if !self.compaction_task_validator.valid_compact_task( @@ -150,7 +155,7 @@ impl CompactionPicker for TierCompactionPicker { return None; } - self.pick_overlapping_level(l0, &level_handlers[0], stats) + self.pick_overlapping_level(l0, &level_handlers[0], levels.vnode_partition_count, stats) } } @@ -257,7 +262,7 @@ pub mod tests { // sub_level_max_compaction_bytes. let mut picker = TierCompactionPicker::new(config); let ret = picker.pick_compaction(&levels, &levels_handler, &mut local_stats); - assert!(ret.is_none()) + assert!(ret.is_none()); } #[test] diff --git a/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs index 9d1e42893a3ae..4f7d4b2bd5da8 100644 --- a/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs @@ -340,6 +340,7 @@ mod test { total_file_size: 0, sub_level_id: 0, uncompressed_file_size: 0, + ..Default::default() }, ]; diff --git a/src/meta/src/hummock/compaction/selector/level_selector.rs b/src/meta/src/hummock/compaction/selector/level_selector.rs index e2b1cc65f3c2a..587bd95743296 100644 --- a/src/meta/src/hummock/compaction/selector/level_selector.rs +++ b/src/meta/src/hummock/compaction/selector/level_selector.rs @@ -238,53 +238,62 @@ impl DynamicLevelSelectorCore { // range at each level, so the number of levels is the most important factor affecting // the read performance. At the same time, the size factor is also added to the score // calculation rule to avoid unbalanced compact task due to large size. - let non_overlapping_score = { - let total_size = levels.l0.as_ref().unwrap().total_file_size - - handlers[0].get_pending_output_file_size(ctx.base_level as u32); - let base_level_size = levels.get_level(ctx.base_level).total_file_size; - let base_level_sst_count = - levels.get_level(ctx.base_level).table_infos.len() as u64; - - // size limit - let non_overlapping_size_score = total_size * SCORE_BASE - / std::cmp::max(self.config.max_bytes_for_level_base, base_level_size); - // level count limit - let non_overlapping_level_count = levels - .l0 - .as_ref() - .unwrap() - .sub_levels - .iter() - .filter(|level| level.level_type() == LevelType::Nonoverlapping) - .count() as u64; - let non_overlapping_level_score = non_overlapping_level_count * SCORE_BASE - / std::cmp::max( - base_level_sst_count / 16, - self.config.level0_sub_level_compact_level_count as u64, - ); - - std::cmp::max(non_overlapping_size_score, non_overlapping_level_score) - }; + let total_size = levels + .l0 + .as_ref() + .unwrap() + .sub_levels + .iter() + .filter(|level| { + level.vnode_partition_count == levels.vnode_partition_count + && level.level_type() == LevelType::Nonoverlapping + }) + .map(|level| level.total_file_size) + .sum::() + - handlers[0].get_pending_output_file_size(ctx.base_level as u32); + let base_level_size = levels.get_level(ctx.base_level).total_file_size; + let base_level_sst_count = levels.get_level(ctx.base_level).table_infos.len() as u64; + + // size limit + let non_overlapping_size_score = total_size * SCORE_BASE + / std::cmp::max(self.config.max_bytes_for_level_base, base_level_size); + // level count limit + let non_overlapping_level_count = levels + .l0 + .as_ref() + .unwrap() + .sub_levels + .iter() + .filter(|level| level.level_type() == LevelType::Nonoverlapping) + .count() as u64; + let non_overlapping_level_score = non_overlapping_level_count * SCORE_BASE + / std::cmp::max( + base_level_sst_count / 16, + self.config.level0_sub_level_compact_level_count as u64, + ); + + let non_overlapping_score = + std::cmp::max(non_overlapping_size_score, non_overlapping_level_score); // Reduce the level num of l0 non-overlapping sub_level - ctx.score_levels.push({ - PickerInfo { + if non_overlapping_size_score > SCORE_BASE { + ctx.score_levels.push(PickerInfo { score: non_overlapping_score + 1, select_level: 0, target_level: ctx.base_level, picker_type: PickerType::ToBase, - } - }); + }); + } - // FIXME: more accurate score calculation algorithm will be introduced (#11903) - ctx.score_levels.push({ - PickerInfo { + if non_overlapping_level_score > SCORE_BASE { + // FIXME: more accurate score calculation algorithm will be introduced (#11903) + ctx.score_levels.push(PickerInfo { score: non_overlapping_score, select_level: 0, target_level: 0, picker_type: PickerType::Intra, - } - }); + }); + } } // The bottommost level can not be input level. diff --git a/src/meta/src/hummock/compaction/selector/mod.rs b/src/meta/src/hummock/compaction/selector/mod.rs index 52d373ccfe9ef..b6900bb5cbbe8 100644 --- a/src/meta/src/hummock/compaction/selector/mod.rs +++ b/src/meta/src/hummock/compaction/selector/mod.rs @@ -126,6 +126,7 @@ pub mod tests { uncompressed_file_size: sst.uncompressed_file_size, sub_level_id: sst.get_sst_id(), table_infos: vec![sst], + ..Default::default() }); } @@ -156,6 +157,7 @@ pub mod tests { sub_level_id, table_infos, uncompressed_file_size, + ..Default::default() }); } @@ -241,6 +243,7 @@ pub mod tests { total_file_size, sub_level_id: 0, uncompressed_file_size, + ..Default::default() } } @@ -263,6 +266,7 @@ pub mod tests { uncompressed_file_size: table.uncompressed_file_size, sub_level_id: idx as u64, table_infos: vec![table], + ..Default::default() }) .collect_vec(), total_file_size, @@ -287,6 +291,7 @@ pub mod tests { .sum::(), sub_level_id: idx as u64, table_infos: table, + ..Default::default() }) .collect_vec(), total_file_size: 0, @@ -321,6 +326,7 @@ pub mod tests { .iter() .map(|sst| sst.uncompressed_file_size) .sum::(), + ..Default::default() }) .collect_vec(), total_file_size: 0, diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index c4059efce8273..a64d6fab3e668 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -952,15 +952,61 @@ impl HummockManager { table_id_to_option.clone(), ); stats.report_to_metrics(compaction_group_id, self.metrics.as_ref()); - let mut compact_task = match compact_task { + let compact_task = match compact_task { None => { return Ok(None); } Some(task) => task, }; - compact_task.watermark = watermark; - compact_task.existing_table_ids = member_table_ids.clone(); + let target_level_id = compact_task.input.target_level; + + let compression_algorithm = match compact_task.compression_algorithm.as_str() { + "Lz4" => 1, + "Zstd" => 2, + _ => 0, + }; + let vnode_partition_count = compact_task.input.vnode_partition_count; + use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; + + let mut compact_task = CompactTask { + input_ssts: compact_task.input.input_levels, + splits: vec![risingwave_pb::hummock::KeyRange::inf()], + watermark, + sorted_output_ssts: vec![], + task_id, + target_level: target_level_id as u32, + // only gc delete keys in last level because there may be older version in more bottom + // level. + gc_delete_keys: target_level_id + == current_version + .get_compaction_group_levels(compaction_group_id) + .levels + .len() + - 1, + base_level: compact_task.base_level as u32, + task_status: TaskStatus::Pending as i32, + compaction_group_id: group_config.group_id, + existing_table_ids: member_table_ids.clone(), + compression_algorithm, + target_file_size: compact_task.target_file_size, + table_options: table_id_to_option + .into_iter() + .filter_map(|(table_id, table_option)| { + if member_table_ids.contains(&table_id) { + return Some((table_id, TableOption::from(&table_option))); + } + + None + }) + .collect(), + current_epoch_time: Epoch::now().0, + compaction_filter_mask: group_config.compaction_config.compaction_filter_mask, + target_sub_level_id: compact_task.input.target_sub_level_id, + task_type: compact_task.compaction_task_type as i32, + split_weight_by_vnode: compact_task.input.vnode_partition_count, + ..Default::default() + }; let is_trivial_reclaim = CompactStatus::is_trivial_reclaim(&compact_task); let is_trivial_move = CompactStatus::is_trivial_move_task(&compact_task); @@ -1010,21 +1056,19 @@ impl HummockManager { compact_task.input_ssts ); } else { - compact_task.table_options = table_id_to_option - .into_iter() - .filter_map(|(table_id, table_option)| { - if compact_task.existing_table_ids.contains(&table_id) { - return Some((table_id, TableOption::from(&table_option))); - } - - None - }) - .collect(); - compact_task.current_epoch_time = Epoch::now().0; - compact_task.compaction_filter_mask = - group_config.compaction_config.compaction_filter_mask; table_to_vnode_partition .retain(|table_id, _| compact_task.existing_table_ids.contains(table_id)); + if current_version + .get_compaction_group_levels(compaction_group_id) + .vnode_partition_count + > 0 + { + for table_id in &compact_task.existing_table_ids { + table_to_vnode_partition + .entry(*table_id) + .or_insert(vnode_partition_count); + } + } compact_task.table_vnode_partition = table_to_vnode_partition; compact_task.table_watermarks = @@ -3237,6 +3281,7 @@ fn gen_version_delta<'a>( level_idx: compact_task.target_level, inserted_table_infos: compact_task.sorted_output_ssts.clone(), l0_sub_level_id: compact_task.target_sub_level_id, + vnode_partition_count: compact_task.split_weight_by_vnode, ..Default::default() })), }; diff --git a/src/meta/src/hummock/metrics_utils.rs b/src/meta/src/hummock/metrics_utils.rs index 84e9d9a9e0372..38377d4eb440e 100644 --- a/src/meta/src/hummock/metrics_utils.rs +++ b/src/meta/src/hummock/metrics_utils.rs @@ -174,9 +174,11 @@ pub fn trigger_sst_stat( { // sub level stat let overlapping_level_label = - build_compact_task_l0_stat_metrics_label(compaction_group_id, true); + build_compact_task_l0_stat_metrics_label(compaction_group_id, true, false); let non_overlap_level_label = - build_compact_task_l0_stat_metrics_label(compaction_group_id, false); + build_compact_task_l0_stat_metrics_label(compaction_group_id, false, false); + let partition_level_label = + build_compact_task_l0_stat_metrics_label(compaction_group_id, true, true); let overlapping_sst_num = current_version .levels @@ -204,6 +206,21 @@ pub fn trigger_sst_stat( }) .unwrap_or(0); + let partition_level_num = current_version + .levels + .get(&compaction_group_id) + .and_then(|level| { + level.l0.as_ref().map(|l0| { + l0.sub_levels + .iter() + .filter(|sub_level| { + sub_level.level_type() == LevelType::Nonoverlapping + && sub_level.vnode_partition_count > 0 + }) + .count() + }) + }) + .unwrap_or(0); metrics .level_sst_num .with_label_values(&[&overlapping_level_label]) @@ -213,6 +230,11 @@ pub fn trigger_sst_stat( .level_sst_num .with_label_values(&[&non_overlap_level_label]) .set(non_overlap_sst_num as i64); + + metrics + .level_sst_num + .with_label_values(&[&partition_level_label]) + .set(partition_level_num as i64); } let previous_time = metrics.time_after_last_observation.load(Ordering::Relaxed); @@ -531,8 +553,11 @@ pub fn build_compact_task_stat_metrics_label( pub fn build_compact_task_l0_stat_metrics_label( compaction_group_id: u64, overlapping: bool, + partition: bool, ) -> String { - if overlapping { + if partition { + format!("cg{}_l0_sub_partition", compaction_group_id) + } else if overlapping { format!("cg{}_l0_sub_overlapping", compaction_group_id) } else { format!("cg{}_l0_sub_non_overlap", compaction_group_id) diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index c494381b4b9f8..064201eb2a7df 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -47,6 +47,7 @@ pub struct GroupDeltasSummary { pub group_destroy: Option, pub group_meta_changes: Vec, pub group_table_change: Option, + pub new_vnode_partition_count: u32, } pub fn summarize_group_deltas(group_deltas: &GroupDeltas) -> GroupDeltasSummary { @@ -59,6 +60,7 @@ pub fn summarize_group_deltas(group_deltas: &GroupDeltas) -> GroupDeltasSummary let mut group_destroy = None; let mut group_meta_changes = vec![]; let mut group_table_change = None; + let mut new_vnode_partition_count = 0; for group_delta in &group_deltas.group_deltas { match group_delta.get_delta_type().unwrap() { @@ -72,6 +74,7 @@ pub fn summarize_group_deltas(group_deltas: &GroupDeltas) -> GroupDeltasSummary insert_sub_level_id = intra_level.l0_sub_level_id; insert_table_infos.extend(intra_level.inserted_table_infos.iter().cloned()); } + new_vnode_partition_count = intra_level.vnode_partition_count; } DeltaType::GroupConstruct(construct_delta) => { assert!(group_construct.is_none()); @@ -103,6 +106,7 @@ pub fn summarize_group_deltas(group_deltas: &GroupDeltas) -> GroupDeltasSummary group_destroy, group_meta_changes, group_table_change, + new_vnode_partition_count, } } @@ -680,6 +684,7 @@ impl Levels { insert_sst_level_id, insert_sub_level_id, insert_table_infos, + new_vnode_partition_count, .. } = summary; @@ -724,9 +729,31 @@ impl Levels { "should find the level to insert into when applying compaction generated delta. sub level idx: {}, removed sst ids: {:?}, sub levels: {:?},", insert_sub_level_id, delete_sst_ids_set, l0.sub_levels.iter().map(|level| level.sub_level_id).collect_vec() ); + if l0.sub_levels[index].table_infos.is_empty() + && self.member_table_ids.len() == 1 + && insert_table_infos.iter().all(|sst| { + sst.table_ids.len() == 1 && sst.table_ids[0] == self.member_table_ids[0] + }) + { + // Only change vnode_partition_count for group which has only one state-table. + // Only change vnode_partition_count for level which update all sst files in this compact task. + l0.sub_levels[index].vnode_partition_count = new_vnode_partition_count; + } level_insert_ssts(&mut l0.sub_levels[index], insert_table_infos); } else { let idx = insert_sst_level_id as usize - 1; + if self.levels[idx].table_infos.is_empty() + && insert_table_infos + .iter() + .all(|sst| sst.table_ids.len() == 1) + { + self.levels[idx].vnode_partition_count = new_vnode_partition_count; + } else if self.levels[idx].vnode_partition_count != 0 + && new_vnode_partition_count == 0 + && self.member_table_ids.len() > 1 + { + self.levels[idx].vnode_partition_count = 0; + } level_insert_ssts(&mut self.levels[idx], insert_table_infos); } } @@ -791,6 +818,7 @@ pub fn build_initial_compaction_group_levels( total_file_size: 0, sub_level_id: 0, uncompressed_file_size: 0, + vnode_partition_count: 0, }); } Levels { @@ -803,6 +831,7 @@ pub fn build_initial_compaction_group_levels( group_id, parent_group_id: StaticCompactionGroupId::NewCompactionGroup as _, member_table_ids: vec![], + vnode_partition_count: compaction_config.split_weight_by_vnode, } } @@ -941,6 +970,7 @@ pub fn new_sub_level( total_file_size, sub_level_id, uncompressed_file_size, + vnode_partition_count: 0, } } diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index 1a32053f544db..7f3810ccb4c49 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -1361,7 +1361,7 @@ pub(crate) mod tests { .last() .unwrap(); assert_eq!(1, output_level_info.table_infos.len()); - assert_eq!(252, output_level_info.table_infos[0].total_key_count); + assert_eq!(254, output_level_info.table_infos[0].total_key_count); } type KeyValue = (FullKey>, HummockValue>); From 3b3d0dec6ba3f724644b95ebbba34ddfd6702afe Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Wed, 10 Jan 2024 18:02:42 +0800 Subject: [PATCH 15/17] feat(frontend): ban jsonb in aggregation stream key (#14442) --- e2e_test/batch/types/jsonb_ord.slt.part | 3 + src/common/src/session_config/mod.rs | 4 + .../testdata/input/jsonb_in_stream_key.yaml | 29 ++++ .../testdata/output/jsonb_in_stream_key.yaml | 34 +++++ src/frontend/src/optimizer/mod.rs | 14 +- .../src/optimizer/plan_node/generic/join.rs | 16 ++- .../src/optimizer/plan_node/logical_join.rs | 5 + .../plan_visitor/jsonb_stream_key_checker.rs | 136 ++++++++++++++++++ .../src/optimizer/plan_visitor/mod.rs | 2 + 9 files changed, 236 insertions(+), 7 deletions(-) create mode 100644 src/frontend/planner_test/tests/testdata/input/jsonb_in_stream_key.yaml create mode 100644 src/frontend/planner_test/tests/testdata/output/jsonb_in_stream_key.yaml create mode 100644 src/frontend/src/optimizer/plan_visitor/jsonb_stream_key_checker.rs diff --git a/e2e_test/batch/types/jsonb_ord.slt.part b/e2e_test/batch/types/jsonb_ord.slt.part index 59dc5406ab718..d146a9513e8b0 100644 --- a/e2e_test/batch/types/jsonb_ord.slt.part +++ b/e2e_test/batch/types/jsonb_ord.slt.part @@ -5,6 +5,9 @@ statement ok SET RW_IMPLICIT_FLUSH TO true; +statement ok +SET rw_streaming_allow_jsonb_in_stream_key TO true; + statement ok values ('{"a":[2, true, "", {}]}'::jsonb), ('1'), ('true'), ('null'), (null), ('[1, true]') order by 1; diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index 067229ebf5fdb..f6f47f11e05bb 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -137,6 +137,10 @@ pub struct ConfigMap { #[parameter(default = false)] streaming_enable_arrangement_backfill: bool, + /// Allow `jsonb` in stream key + #[parameter(default = false, rename = "rw_streaming_allow_jsonb_in_stream_key")] + streaming_allow_jsonb_in_stream_key: bool, + /// Enable join ordering for streaming and batch queries. Defaults to true. #[parameter(default = true, rename = "rw_enable_join_ordering")] enable_join_ordering: bool, diff --git a/src/frontend/planner_test/tests/testdata/input/jsonb_in_stream_key.yaml b/src/frontend/planner_test/tests/testdata/input/jsonb_in_stream_key.yaml new file mode 100644 index 0000000000000..11988174503f7 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/jsonb_in_stream_key.yaml @@ -0,0 +1,29 @@ +- name: jsonb in group by + sql: | + create table t1 (v1 jsonb, v2 int); + create table t2 (v3 jsonb, v4 int); + select v2 from t1 group by v2, v1; + expected_outputs: + - stream_error +- name: jsonb in union + sql: | + create table t1 (v1 jsonb, v2 int); + create table t2 (v3 jsonb, v4 int); + select v1, v2 from t1 union select v3, v4 from t2; + expected_outputs: + - stream_error +- name: jsonb in distinct + sql: | + create table t1 (v1 jsonb, v2 int); + select distinct v1 from t1; + expected_outputs: + - stream_error +- name: jsonb in TopN by group + sql: | + create table t1 (v1 jsonb, v2 int); + SELECT v1 FROM ( + SELECT v1, rank() OVER (PARTITION BY v1 ORDER BY v2) AS rank + FROM t1) + WHERE rank <= 2; + expected_outputs: + - stream_error diff --git a/src/frontend/planner_test/tests/testdata/output/jsonb_in_stream_key.yaml b/src/frontend/planner_test/tests/testdata/output/jsonb_in_stream_key.yaml new file mode 100644 index 0000000000000..9cf9c5f8853d1 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/jsonb_in_stream_key.yaml @@ -0,0 +1,34 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. +- name: jsonb in group by + sql: | + create table t1 (v1 jsonb, v2 int); + create table t2 (v3 jsonb, v4 int); + select v2 from t1 group by v2, v1; + stream_error: |- + Not supported: JSONB column "aggregation group key" should not be in the t1.v1. + HINT: Using JSONB columns as part of the join or aggregation keys can severely impair performance. If you intend to proceed, force to enable it with: `set rw_streaming_allow_jsonb_in_stream_key to true` +- name: jsonb in union + sql: | + create table t1 (v1 jsonb, v2 int); + create table t2 (v3 jsonb, v4 int); + select v1, v2 from t1 union select v3, v4 from t2; + stream_error: |- + Not supported: JSONB column "field" should not be in the t1.v1. + HINT: Using JSONB columns as part of the join or aggregation keys can severely impair performance. If you intend to proceed, force to enable it with: `set rw_streaming_allow_jsonb_in_stream_key to true` +- name: jsonb in distinct + sql: | + create table t1 (v1 jsonb, v2 int); + select distinct v1 from t1; + stream_error: |- + Not supported: JSONB column "aggregation group key" should not be in the t1.v1. + HINT: Using JSONB columns as part of the join or aggregation keys can severely impair performance. If you intend to proceed, force to enable it with: `set rw_streaming_allow_jsonb_in_stream_key to true` +- name: jsonb in TopN by group + sql: | + create table t1 (v1 jsonb, v2 int); + SELECT v1 FROM ( + SELECT v1, rank() OVER (PARTITION BY v1 ORDER BY v2) AS rank + FROM t1) + WHERE rank <= 2; + stream_error: |- + Not supported: JSONB column "over window partition key" should not be in the t1.v1. + HINT: Using JSONB columns as part of the join or aggregation keys can severely impair performance. If you intend to proceed, force to enable it with: `set rw_streaming_allow_jsonb_in_stream_key to true` diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 5f32605b14184..b572539ecdd84 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -65,7 +65,7 @@ use self::plan_node::{ }; #[cfg(debug_assertions)] use self::plan_visitor::InputRefValidator; -use self::plan_visitor::{has_batch_exchange, CardinalityVisitor}; +use self::plan_visitor::{has_batch_exchange, CardinalityVisitor, StreamKeyChecker}; use self::property::{Cardinality, RequiredDist}; use self::rule::*; use crate::catalog::table_catalog::{TableType, TableVersion}; @@ -417,6 +417,18 @@ impl PlanRoot { let plan = match self.plan.convention() { Convention::Logical => { + if !ctx + .session_ctx() + .config() + .streaming_allow_jsonb_in_stream_key() + && let Some(err) = StreamKeyChecker.visit(self.plan.clone()) + { + return Err(ErrorCode::NotSupported( + err, + "Using JSONB columns as part of the join or aggregation keys can severely impair performance. \ + If you intend to proceed, force to enable it with: `set rw_streaming_allow_jsonb_in_stream_key to true`".to_string(), + ).into()); + } let plan = self.gen_optimized_logical_plan_for_stream()?; let (plan, out_col_change) = { diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index b6ab9fe6de5d5..b49baf2a75f82 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -46,7 +46,7 @@ pub(crate) fn has_repeated_element(slice: &[usize]) -> bool { (1..slice.len()).any(|i| slice[i..].contains(&slice[i - 1])) } -impl Join { +impl Join { pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) { self.on = self.on.clone().rewrite_expr(r); } @@ -55,6 +55,13 @@ impl Join { self.on.visit_expr(v); } + pub fn eq_indexes(&self) -> Vec<(usize, usize)> { + let left_len = self.left.schema().len(); + let right_len = self.right.schema().len(); + let eq_predicate = EqJoinPredicate::create(left_len, right_len, self.on.clone()); + eq_predicate.eq_indexes() + } + pub fn new( left: PlanRef, right: PlanRef, @@ -169,10 +176,7 @@ impl GenericPlanNode for Join { } fn stream_key(&self) -> Option> { - let left_len = self.left.schema().len(); - let right_len = self.right.schema().len(); - let eq_predicate = EqJoinPredicate::create(left_len, right_len, self.on.clone()); - + let eq_indexes = self.eq_indexes(); let left_pk = self.left.stream_key()?; let right_pk = self.right.stream_key()?; let l2i = self.l2i_col_mapping(); @@ -197,7 +201,7 @@ impl GenericPlanNode for Join { let either_or_both = self.add_which_join_key_to_pk(); - for (lk, rk) in eq_predicate.eq_indexes() { + for (lk, rk) in eq_indexes { match either_or_both { EitherOrBoth::Left(_) => { // Remove right-side join-key column it from pk_indices. diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 5365eb3642b79..dd555e5e3a1c0 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -151,6 +151,11 @@ impl LogicalJoin { self.core.join_type } + /// Get the eq join key of the logical join. + pub fn eq_indexes(&self) -> Vec<(usize, usize)> { + self.core.eq_indexes() + } + /// Get the output indices of the logical join. pub fn output_indices(&self) -> &Vec { &self.core.output_indices diff --git a/src/frontend/src/optimizer/plan_visitor/jsonb_stream_key_checker.rs b/src/frontend/src/optimizer/plan_visitor/jsonb_stream_key_checker.rs new file mode 100644 index 0000000000000..d0a4358b4582a --- /dev/null +++ b/src/frontend/src/optimizer/plan_visitor/jsonb_stream_key_checker.rs @@ -0,0 +1,136 @@ +// 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_common::catalog::{Field, FieldDisplay}; +use risingwave_common::types::DataType; + +use super::{DefaultBehavior, Merge}; +use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::{PlanNode, *}; +use crate::optimizer::plan_visitor::PlanVisitor; + +#[derive(Debug, Clone, Default)] +pub struct StreamKeyChecker; + +impl StreamKeyChecker { + fn visit_inputs(&mut self, plan: &impl PlanNode) -> Option { + let results = plan.inputs().into_iter().map(|input| self.visit(input)); + Self::default_behavior().apply(results) + } + + fn err_msg(target: &str, field: &Field) -> String { + format!( + "JSONB column \"{}\" should not be in the {}.", + target, + FieldDisplay(field) + ) + } +} + +impl PlanVisitor for StreamKeyChecker { + type Result = Option; + + type DefaultBehavior = impl DefaultBehavior; + + fn default_behavior() -> Self::DefaultBehavior { + Merge(|a: Option, b| a.or(b)) + } + + fn visit_logical_dedup(&mut self, plan: &LogicalDedup) -> Self::Result { + let input = plan.input(); + let schema = input.schema(); + let data_types = schema.data_types(); + for idx in plan.dedup_cols() { + if data_types[*idx] == DataType::Jsonb { + return Some(StreamKeyChecker::err_msg("distinct key", &schema[*idx])); + } + } + self.visit_inputs(plan) + } + + fn visit_logical_top_n(&mut self, plan: &LogicalTopN) -> Self::Result { + let input = plan.input(); + let schema = input.schema(); + let data_types = schema.data_types(); + for idx in plan.group_key() { + if data_types[*idx] == DataType::Jsonb { + return Some(StreamKeyChecker::err_msg("TopN group key", &schema[*idx])); + } + } + for idx in plan + .topn_order() + .column_orders + .iter() + .map(|c| c.column_index) + { + if data_types[idx] == DataType::Jsonb { + return Some(StreamKeyChecker::err_msg("TopN order key", &schema[idx])); + } + } + self.visit_inputs(plan) + } + + fn visit_logical_union(&mut self, plan: &LogicalUnion) -> Self::Result { + if !plan.all() { + for field in &plan.inputs()[0].schema().fields { + if field.data_type() == DataType::Jsonb { + return Some(StreamKeyChecker::err_msg("field", field)); + } + } + } + self.visit_inputs(plan) + } + + fn visit_logical_agg(&mut self, plan: &LogicalAgg) -> Self::Result { + let input = plan.input(); + let schema = input.schema(); + let data_types = schema.data_types(); + for idx in plan.group_key().indices() { + if data_types[idx] == DataType::Jsonb { + return Some(StreamKeyChecker::err_msg( + "aggregation group key", + &schema[idx], + )); + } + } + self.visit_inputs(plan) + } + + fn visit_logical_over_window(&mut self, plan: &LogicalOverWindow) -> Self::Result { + let input = plan.input(); + let schema = input.schema(); + let data_types = schema.data_types(); + + for func in plan.window_functions() { + for idx in func.partition_by.iter().map(|e| e.index()) { + if data_types[idx] == DataType::Jsonb { + return Some(StreamKeyChecker::err_msg( + "over window partition key", + &schema[idx], + )); + } + } + + for idx in func.order_by.iter().map(|c| c.column_index) { + if data_types[idx] == DataType::Jsonb { + return Some(StreamKeyChecker::err_msg( + "over window order by key", + &schema[idx], + )); + } + } + } + self.visit_inputs(plan) + } +} diff --git a/src/frontend/src/optimizer/plan_visitor/mod.rs b/src/frontend/src/optimizer/plan_visitor/mod.rs index ffb3a814f2ccc..fa9170cbf30fb 100644 --- a/src/frontend/src/optimizer/plan_visitor/mod.rs +++ b/src/frontend/src/optimizer/plan_visitor/mod.rs @@ -37,6 +37,8 @@ mod side_effect_visitor; pub use side_effect_visitor::*; mod cardinality_visitor; pub use cardinality_visitor::*; +mod jsonb_stream_key_checker; +pub use jsonb_stream_key_checker::*; use crate::for_all_plan_nodes; use crate::optimizer::plan_node::*; From 50d111490b921430aef3f6a3e220e66466606e9e Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Wed, 10 Jan 2024 19:11:12 +0800 Subject: [PATCH 16/17] fix: update downstream table's upstream_actor_id for sinking into table (#14487) --- src/meta/src/rpc/ddl_controller.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 72d0086ba2fac..c613385735125 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -1047,6 +1047,11 @@ impl DdlController { } } + // update downstream actors' upstream_actor_id and upstream_fragment_id + for actor in &mut union_fragment.actors { + actor.upstream_actor_id.extend(sink_actor_ids.clone()); + } + union_fragment .upstream_fragment_ids .push(upstream_fragment_id); From cf9a2dbccc6c53167a15436ab947f99f4f5734cc Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Wed, 10 Jan 2024 19:19:18 +0800 Subject: [PATCH 17/17] feat: support scaling table fragments connected by hash shuffle (#14485) --- src/meta/src/manager/catalog/fragment.rs | 81 +++++++++++++++++------- src/meta/src/stream/scale.rs | 3 +- 2 files changed, 58 insertions(+), 26 deletions(-) diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index 51778c7a71a01..d359c3fa453c9 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -1120,9 +1120,20 @@ impl FragmentManager { .filter(|t| t.fragment_ids().any(|f| reschedules.contains_key(&f))) .map(|t| t.table_id()) .collect_vec(); + + let fragment_id_to_table_id: HashMap<_, _> = map + .iter() + .flat_map(|(table_id, table)| { + table + .fragment_ids() + .map(|fragment_id| (fragment_id, *table_id)) + }) + .collect(); + let mut table_fragments = BTreeMapTransaction::new(map); let mut fragment_mapping_to_notify = vec![]; + // First step, update fragment itself for table_id in to_update_table_fragments { // Takes out the reschedules of the fragments in this table. let reschedules = reschedules @@ -1137,20 +1148,17 @@ impl FragmentManager { let mut table_fragment = table_fragments.get_mut(table_id).unwrap(); - for (fragment_id, reschedule) in reschedules { + for (fragment_id, reschedule) in &reschedules { let Reschedule { added_actors, removed_actors, vnode_bitmap_updates, - upstream_fragment_dispatcher_ids, - upstream_dispatcher_mapping, - downstream_fragment_ids, actor_splits, + .. } = reschedule; - // First step, update self fragment // Add actors to this fragment: set the state to `Running`. - for actor_id in &added_actors { + for actor_id in added_actors { table_fragment .actor_status .get_mut(actor_id) @@ -1166,10 +1174,10 @@ impl FragmentManager { table_fragment.actor_splits.remove(actor_id); } - table_fragment.actor_splits.extend(actor_splits); + table_fragment.actor_splits.extend(actor_splits.clone()); let actor_status = table_fragment.actor_status.clone(); - let fragment = table_fragment.fragments.get_mut(&fragment_id).unwrap(); + let fragment = table_fragment.fragments.get_mut(fragment_id).unwrap(); fragment .actors @@ -1215,21 +1223,38 @@ impl FragmentManager { // Notify fragment mapping to frontend nodes. let fragment_mapping = FragmentParallelUnitMapping { - fragment_id: fragment_id as FragmentId, + fragment_id: *fragment_id as FragmentId, mapping: Some(vnode_mapping), }; fragment_mapping_to_notify.push(fragment_mapping); + } + + // Second step, update upstream fragments & downstream fragments + for (fragment_id, reschedule) in &reschedules { + let Reschedule { + upstream_fragment_dispatcher_ids, + upstream_dispatcher_mapping, + downstream_fragment_ids, + added_actors, + removed_actors, + .. + } = reschedule; + + let removed_actor_ids: HashSet<_> = removed_actors.iter().cloned().collect(); - // Second step, update upstream fragments // Update the dispatcher of the upstream fragments. for (upstream_fragment_id, dispatcher_id) in upstream_fragment_dispatcher_ids { - // here we assume the upstream fragment is in the same streaming job as this - // fragment. Cross-table references only occur in the case - // of StreamScan fragment, and the scale of StreamScan fragment does not introduce updates - // to the upstream Fragment (because of NoShuffle) - let upstream_fragment = table_fragment + let upstream_table_id = fragment_id_to_table_id + .get(upstream_fragment_id) + .expect("upstream fragment must exist"); + + // After introducing arrangement backfill and sink into table, two tables might be connected via operators outside of the NO_SHUFFLE. + let mut upstream_table_fragment = + table_fragments.get_mut(*upstream_table_id).unwrap(); + + let upstream_fragment = upstream_table_fragment .fragments - .get_mut(&upstream_fragment_id) + .get_mut(upstream_fragment_id) .unwrap(); for upstream_actor in &mut upstream_fragment.actors { @@ -1238,7 +1263,7 @@ impl FragmentManager { } for dispatcher in &mut upstream_actor.dispatcher { - if dispatcher.dispatcher_id == dispatcher_id { + if dispatcher.dispatcher_id == *dispatcher_id { if let DispatcherType::Hash = dispatcher.r#type() { dispatcher.hash_mapping = upstream_dispatcher_mapping .as_ref() @@ -1248,7 +1273,7 @@ impl FragmentManager { update_actors( dispatcher.downstream_actor_id.as_mut(), &removed_actor_ids, - &added_actors, + added_actors, ); } } @@ -1256,11 +1281,19 @@ impl FragmentManager { } // Update the merge executor of the downstream fragment. - for &downstream_fragment_id in &downstream_fragment_ids { - let downstream_fragment = table_fragment + for downstream_fragment_id in downstream_fragment_ids { + let downstream_table_id = fragment_id_to_table_id + .get(downstream_fragment_id) + .expect("downstream fragment must exist"); + + let mut downstream_table_fragment = + table_fragments.get_mut(*downstream_table_id).unwrap(); + + let downstream_fragment = downstream_table_fragment .fragments - .get_mut(&downstream_fragment_id) + .get_mut(downstream_fragment_id) .unwrap(); + for downstream_actor in &mut downstream_fragment.actors { if new_created_actors.contains(&downstream_actor.actor_id) { continue; @@ -1269,15 +1302,15 @@ impl FragmentManager { update_actors( downstream_actor.upstream_actor_id.as_mut(), &removed_actor_ids, - &added_actors, + added_actors, ); if let Some(node) = downstream_actor.nodes.as_mut() { update_merge_node_upstream( node, - &fragment_id, + fragment_id, &removed_actor_ids, - &added_actors, + added_actors, ); } } diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index 376d8ad69e59b..fc0a7ef55b8f1 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -138,7 +138,7 @@ impl RescheduleContext { self.actor_status .get(actor_id) .and_then(|actor_status| actor_status.parallel_unit.as_ref()) - .ok_or_else(|| anyhow!("could not found ParallelUnit for {}", actor_id).into()) + .ok_or_else(|| anyhow!("could not found parallel unit for actor {}", actor_id).into()) } fn parallel_unit_id_to_worker( @@ -1048,7 +1048,6 @@ impl ScaleController { if new_created_actors.contains_key(downstream_actor_id) { continue; } - let downstream_worker_id = ctx .actor_id_to_parallel_unit(downstream_actor_id)? .worker_node_id;