From f0ece4520ff33d07a7ae55af9621d8af69415a4d Mon Sep 17 00:00:00 2001 From: Dylan Date: Tue, 12 Nov 2024 21:47:50 +0800 Subject: [PATCH 01/16] fix(iceberg): change iceberg rest catalog default implementation to java (#19357) --- .../src/connector_common/iceberg/mod.rs | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/src/connector/src/connector_common/iceberg/mod.rs b/src/connector/src/connector_common/iceberg/mod.rs index bf96e474eee80..77a18368b0000 100644 --- a/src/connector/src/connector_common/iceberg/mod.rs +++ b/src/connector/src/connector_common/iceberg/mod.rs @@ -258,7 +258,6 @@ mod v1 { let catalog_type = self.catalog_type().to_string(); - iceberg_configs.insert(CATALOG_TYPE.to_string(), catalog_type.clone()); iceberg_configs.insert(CATALOG_NAME.to_string(), self.catalog_name()); match catalog_type.as_str() { @@ -267,14 +266,16 @@ mod v1 { format!("iceberg.catalog.{}.warehouse", self.catalog_name()), self.warehouse_path.clone(), ); + iceberg_configs.insert(CATALOG_TYPE.to_string(), "storage".into()); } - "rest" => { + "rest_rust" => { let uri = self .catalog_uri .clone() .with_context(|| "`catalog.uri` must be set in rest catalog".to_string())?; iceberg_configs .insert(format!("iceberg.catalog.{}.uri", self.catalog_name()), uri); + iceberg_configs.insert(CATALOG_TYPE.to_string(), "rest".into()); } _ => { bail!( @@ -351,7 +352,7 @@ mod v1 { java_catalog_props: &HashMap, ) -> ConnectorResult { match self.catalog_type() { - "storage" | "rest" => { + "storage" | "rest_rust" => { let iceberg_configs = self.build_iceberg_configs()?; let catalog = load_catalog(&iceberg_configs).await?; Ok(catalog) @@ -359,7 +360,8 @@ mod v1 { catalog_type if catalog_type == "hive" || catalog_type == "jdbc" - || catalog_type == "glue" => + || catalog_type == "glue" + || catalog_type == "rest" => { // Create java catalog let (base_catalog_config, java_catalog_props) = @@ -368,6 +370,7 @@ mod v1 { "hive" => "org.apache.iceberg.hive.HiveCatalog", "jdbc" => "org.apache.iceberg.jdbc.JdbcCatalog", "glue" => "org.apache.iceberg.aws.glue.GlueCatalog", + "rest" => "org.apache.iceberg.rest.RESTCatalog", _ => unreachable!(), }; @@ -444,7 +447,7 @@ mod v2 { let catalog = storage_catalog::StorageCatalog::new(config)?; Ok(Arc::new(catalog)) } - "rest" => { + "rest_rust" => { let mut iceberg_configs = HashMap::new(); if let Some(region) = &self.region { iceberg_configs.insert(S3_REGION.to_string(), region.clone().to_string()); @@ -512,13 +515,18 @@ mod v2 { let catalog = iceberg_catalog_glue::GlueCatalog::new(config).await?; Ok(Arc::new(catalog)) } - catalog_type if catalog_type == "hive" || catalog_type == "jdbc" => { + catalog_type + if catalog_type == "hive" + || catalog_type == "jdbc" + || catalog_type == "rest" => + { // Create java catalog let (base_catalog_config, java_catalog_props) = self.build_jni_catalog_configs(java_catalog_props)?; let catalog_impl = match catalog_type { "hive" => "org.apache.iceberg.hive.HiveCatalog", "jdbc" => "org.apache.iceberg.jdbc.JdbcCatalog", + "rest" => "org.apache.iceberg.rest.RESTCatalog", _ => unreachable!(), }; From c0b5e641f0a569a8551fa2f13e1e51c7daee1fd2 Mon Sep 17 00:00:00 2001 From: Dylan Date: Wed, 13 Nov 2024 10:55:38 +0800 Subject: [PATCH 02/16] fix(binder): report error on update query with subquery on the set clause (#19305) --- .../planner_test/tests/testdata/input/update.yaml | 7 +++++++ .../planner_test/tests/testdata/output/update.yaml | 6 ++++++ src/frontend/src/binder/update.rs | 13 +++++++------ 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/update.yaml b/src/frontend/planner_test/tests/testdata/input/update.yaml index 2322f68ad95e0..65c0f47eb4cd4 100644 --- a/src/frontend/planner_test/tests/testdata/input/update.yaml +++ b/src/frontend/planner_test/tests/testdata/input/update.yaml @@ -98,3 +98,10 @@ update t set a = a + 1; expected_outputs: - batch_distributed_plan +- name: update table with subquery in the set clause + sql: | + create table t1 (v1 int primary key, v2 int); + create table t2 (v1 int primary key, v2 int); + update t1 set v1 = (select v1 from t2 where t1.v2 = t2.v2); + expected_outputs: + - binder_error diff --git a/src/frontend/planner_test/tests/testdata/output/update.yaml b/src/frontend/planner_test/tests/testdata/output/update.yaml index 884b091cf8af7..eae928bb858a7 100644 --- a/src/frontend/planner_test/tests/testdata/output/update.yaml +++ b/src/frontend/planner_test/tests/testdata/output/update.yaml @@ -165,3 +165,9 @@ └─BatchUpdate { table: t, exprs: [($0 + 1:Int32), $1, $2] } └─BatchExchange { order: [], dist: HashShard(t.a, t.b, t._row_id) } └─BatchScan { table: t, columns: [t.a, t.b, t._row_id], distribution: UpstreamHashShard(t._row_id) } +- name: update table with subquery in the set clause + sql: | + create table t1 (v1 int primary key, v2 int); + create table t2 (v1 int primary key, v2 int); + update t1 set v1 = (select v1 from t2 where t1.v2 = t2.v2); + binder_error: 'Bind error: subquery on the right side of assignment is unsupported' diff --git a/src/frontend/src/binder/update.rs b/src/frontend/src/binder/update.rs index cbe51f9ec69bd..6775c2173e00b 100644 --- a/src/frontend/src/binder/update.rs +++ b/src/frontend/src/binder/update.rs @@ -17,7 +17,6 @@ use std::collections::{BTreeMap, HashMap}; use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::bail_not_implemented; use risingwave_common::catalog::{Schema, TableVersionId}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_sqlparser::ast::{Assignment, AssignmentValue, Expr, ObjectName, SelectItem}; @@ -129,15 +128,17 @@ impl Binder { for Assignment { id, value } in assignments { // FIXME: Parsing of `id` is not strict. It will even treat `a.b` as `(a, b)`. let assignments = match (id.as_slice(), value) { + // _ = (subquery) + (_ids, AssignmentValue::Expr(Expr::Subquery(_))) => { + return Err(ErrorCode::BindError( + "subquery on the right side of assignment is unsupported".to_owned(), + ) + .into()) + } // col = expr ([id], value) => { vec![(id.clone(), value)] } - - // (col1, col2) = (subquery) - (_ids, AssignmentValue::Expr(Expr::Subquery(_))) => { - bail_not_implemented!("subquery on the right side of multi-assignment"); - } // (col1, col2) = (expr1, expr2) // TODO: support `DEFAULT` in multiple assignments (ids, AssignmentValue::Expr(Expr::Row(values))) if ids.len() == values.len() => id From c93b92bab4c3c5d6ae2491de384dda685e9df88f Mon Sep 17 00:00:00 2001 From: Li0k Date: Wed, 13 Nov 2024 11:36:35 +0800 Subject: [PATCH 03/16] feat(compaction): Introducing configuration for compactor preload SstableMeta (#19218) --- src/common/src/config.rs | 10 ++++++++++ src/config/docs.md | 1 + src/config/example.toml | 1 + src/storage/src/hummock/compactor/compaction_utils.rs | 3 +-- src/storage/src/opts.rs | 4 ++++ 5 files changed, 17 insertions(+), 2 deletions(-) diff --git a/src/common/src/config.rs b/src/common/src/config.rs index c3fc78919a3ef..64d7675903ec4 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -871,6 +871,12 @@ pub struct StorageConfig { #[serde(default = "default::storage::compactor_max_overlap_sst_count")] pub compactor_max_overlap_sst_count: usize, + /// The maximum number of meta files that can be preloaded. + /// If the number of meta files exceeds this value, the compactor will try to compute parallelism only through `SstableInfo`, no longer preloading `SstableMeta`. + /// This is to prevent the compactor from consuming too much memory, but it may cause the compactor to be less efficient. + #[serde(default = "default::storage::compactor_max_preload_meta_file_count")] + pub compactor_max_preload_meta_file_count: usize, + /// Object storage configuration /// 1. General configuration /// 2. Some special configuration of Backend @@ -1795,6 +1801,10 @@ pub mod default { 64 } + pub fn compactor_max_preload_meta_file_count() -> usize { + 32 + } + // deprecated pub fn table_info_statistic_history_times() -> usize { 240 diff --git a/src/config/docs.md b/src/config/docs.md index 4a25867ed63c1..bfe6a2fb5429c 100644 --- a/src/config/docs.md +++ b/src/config/docs.md @@ -125,6 +125,7 @@ This page is automatically generated by `./risedev generate-example-config` | compactor_fast_max_compact_task_size | | 2147483648 | | compactor_iter_max_io_retry_times | | 8 | | compactor_max_overlap_sst_count | | 64 | +| compactor_max_preload_meta_file_count | The maximum number of meta files that can be preloaded. If the number of meta files exceeds this value, the compactor will try to compute parallelism only through `SstableInfo`, no longer preloading `SstableMeta`. This is to prevent the compactor from consuming too much memory, but it may cause the compactor to be less efficient. | 32 | | compactor_max_sst_key_count | | 2097152 | | compactor_max_sst_size | | 536870912 | | compactor_max_task_multiplier | Compactor calculates the maximum number of tasks that can be executed on the node based on `worker_num` and `compactor_max_task_multiplier`. `max_pull_task_count` = `worker_num` * `compactor_max_task_multiplier` | 3.0 | diff --git a/src/config/example.toml b/src/config/example.toml index 1d540b429a876..7a01ff5254e7f 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -176,6 +176,7 @@ compactor_iter_max_io_retry_times = 8 table_info_statistic_history_times = 240 mem_table_spill_threshold = 4194304 compactor_max_overlap_sst_count = 64 +compactor_max_preload_meta_file_count = 32 time_travel_version_cache_capacity = 32 [storage.cache.block_cache_eviction] diff --git a/src/storage/src/hummock/compactor/compaction_utils.rs b/src/storage/src/hummock/compactor/compaction_utils.rs index 5e22d3e45701b..73b178d73e802 100644 --- a/src/storage/src/hummock/compactor/compaction_utils.rs +++ b/src/storage/src/hummock/compactor/compaction_utils.rs @@ -232,10 +232,9 @@ pub async fn generate_splits( context: &CompactorContext, max_sub_compaction: u32, ) -> HummockResult> { - const MAX_FILE_COUNT: usize = 32; let parallel_compact_size = (context.storage_opts.parallel_compact_size_mb as u64) << 20; if compaction_size > parallel_compact_size { - if sstable_infos.len() > MAX_FILE_COUNT { + if sstable_infos.len() > context.storage_opts.compactor_max_preload_meta_file_count { return Ok(generate_splits_fast( sstable_infos, compaction_size, diff --git a/src/storage/src/opts.rs b/src/storage/src/opts.rs index 14500f7c5113e..e41e07dafc44a 100644 --- a/src/storage/src/opts.rs +++ b/src/storage/src/opts.rs @@ -140,6 +140,9 @@ pub struct StorageOpts { pub compactor_max_overlap_sst_count: usize, + /// The maximum number of meta files that can be preloaded. + pub compactor_max_preload_meta_file_count: usize, + pub object_store_config: ObjectStoreConfig, pub time_travel_version_cache_capacity: u64, } @@ -243,6 +246,7 @@ impl From<(&RwConfig, &SystemParamsReader, &StorageMemoryConfig)> for StorageOpt .compactor_concurrent_uploading_sst_count, time_travel_version_cache_capacity: c.storage.time_travel_version_cache_capacity, compactor_max_overlap_sst_count: c.storage.compactor_max_overlap_sst_count, + compactor_max_preload_meta_file_count: c.storage.compactor_max_preload_meta_file_count, } } } From c21a7710fe014607a728c0a94d489291874e3162 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 13 Nov 2024 12:02:43 +0800 Subject: [PATCH 04/16] fix(agg): fix `first_value` and `last_value` to not ignore NULLs (#19332) Signed-off-by: Richard Chien --- .../impl/src/aggregate/first_last_value.rs | 88 +++++++++++++++++++ src/expr/impl/src/aggregate/general.rs | 19 ---- src/expr/impl/src/aggregate/mod.rs | 1 + src/tests/sqlsmith/src/sql_gen/scalar.rs | 14 +-- 4 files changed, 98 insertions(+), 24 deletions(-) create mode 100644 src/expr/impl/src/aggregate/first_last_value.rs diff --git a/src/expr/impl/src/aggregate/first_last_value.rs b/src/expr/impl/src/aggregate/first_last_value.rs new file mode 100644 index 0000000000000..841442148f722 --- /dev/null +++ b/src/expr/impl/src/aggregate/first_last_value.rs @@ -0,0 +1,88 @@ +// 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::types::{Datum, ScalarRefImpl}; +use risingwave_common_estimate_size::EstimateSize; +use risingwave_expr::aggregate; +use risingwave_expr::aggregate::AggStateDyn; + +/// Note that different from `min` and `max`, `first_value` doesn't ignore `NULL` values. +/// +/// ```slt +/// statement ok +/// create table t(v1 int, ts int); +/// +/// statement ok +/// insert into t values (null, 1), (2, 2), (null, 3); +/// +/// query I +/// select first_value(v1 order by ts) from t; +/// ---- +/// NULL +/// +/// statement ok +/// drop table t; +/// ``` +#[aggregate("first_value(any) -> any")] +fn first_value(state: &mut FirstValueState, input: Option>) { + if state.0.is_none() { + state.0 = Some(input.map(|x| x.into_scalar_impl())); + } +} + +#[derive(Debug, Clone, Default, EstimateSize)] +struct FirstValueState(Option); + +impl AggStateDyn for FirstValueState {} + +impl From<&FirstValueState> for Datum { + fn from(state: &FirstValueState) -> Self { + if let Some(state) = &state.0 { + state.clone() + } else { + None + } + } +} + +/// Note that different from `min` and `max`, `last_value` doesn't ignore `NULL` values. +/// +/// ```slt +/// statement ok +/// create table t(v1 int, ts int); +/// +/// statement ok +/// insert into t values (null, 1), (2, 2), (null, 3); +/// +/// query I +/// select last_value(v1 order by ts) from t; +/// ---- +/// NULL +/// +/// statement ok +/// drop table t; +/// ``` +#[aggregate("last_value(*) -> auto", state = "ref")] // TODO(rc): `last_value(any) -> any` +fn last_value(_: Option, input: Option) -> Option { + input +} + +#[aggregate("internal_last_seen_value(*) -> auto", state = "ref", internal)] +fn internal_last_seen_value(state: T, input: T, retract: bool) -> T { + if retract { + state + } else { + input + } +} diff --git a/src/expr/impl/src/aggregate/general.rs b/src/expr/impl/src/aggregate/general.rs index 0c94312335b4b..daaea5e782fd1 100644 --- a/src/expr/impl/src/aggregate/general.rs +++ b/src/expr/impl/src/aggregate/general.rs @@ -124,25 +124,6 @@ fn max(state: T, input: T) -> T { state.max(input) } -#[aggregate("first_value(*) -> auto", state = "ref")] -fn first_value(state: T, _: T) -> T { - state -} - -#[aggregate("last_value(*) -> auto", state = "ref")] -fn last_value(_: T, input: T) -> T { - input -} - -#[aggregate("internal_last_seen_value(*) -> auto", state = "ref", internal)] -fn internal_last_seen_value(state: T, input: T, retract: bool) -> T { - if retract { - state - } else { - input - } -} - /// Note the following corner cases: /// /// ```slt diff --git a/src/expr/impl/src/aggregate/mod.rs b/src/expr/impl/src/aggregate/mod.rs index 349574018fedf..881465b4cf82f 100644 --- a/src/expr/impl/src/aggregate/mod.rs +++ b/src/expr/impl/src/aggregate/mod.rs @@ -20,6 +20,7 @@ mod bit_or; mod bit_xor; mod bool_and; mod bool_or; +mod first_last_value; mod general; mod jsonb_agg; mod mode; diff --git a/src/tests/sqlsmith/src/sql_gen/scalar.rs b/src/tests/sqlsmith/src/sql_gen/scalar.rs index 62cd7218dcc90..a532f6138c596 100644 --- a/src/tests/sqlsmith/src/sql_gen/scalar.rs +++ b/src/tests/sqlsmith/src/sql_gen/scalar.rs @@ -81,11 +81,15 @@ impl SqlGenerator<'_, R> { data_type: AstDataType::SmallInt, value: self.gen_int(i16::MIN as isize, i16::MAX as isize), })), - T::Varchar => Expr::Value(Value::SingleQuotedString( - (0..10) - .map(|_| self.rng.sample(Alphanumeric) as char) - .collect(), - )), + T::Varchar => Expr::Cast { + // since we are generating random scalar literal, we should cast it to avoid unknown type + expr: Box::new(Expr::Value(Value::SingleQuotedString( + (0..10) + .map(|_| self.rng.sample(Alphanumeric) as char) + .collect(), + ))), + data_type: AstDataType::Varchar, + }, T::Decimal => Expr::Nested(Box::new(Expr::Value(Value::Number(self.gen_float())))), T::Float64 => Expr::Nested(Box::new(Expr::TypedString { data_type: AstDataType::Float(None), From 0ddb6eb17b514aef65f1625ec46087ca212711c1 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 13 Nov 2024 14:53:28 +0800 Subject: [PATCH 05/16] refactor(barrier): not cache actor failure in local barrier worker (#19367) --- src/stream/src/task/barrier_manager.rs | 64 +++++--------------------- 1 file changed, 12 insertions(+), 52 deletions(-) diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 0b5f6d6178e67..0bfdcdd59117c 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -12,9 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeSet, HashMap}; +use std::collections::BTreeSet; use std::fmt::Display; use std::future::pending; +use std::iter::once; use std::sync::Arc; use std::time::Duration; @@ -260,9 +261,6 @@ pub(super) struct LocalBarrierWorker { /// Current barrier collection state. pub(super) state: ManagedBarrierState, - /// Record all unexpected exited actors. - failure_actors: HashMap, - control_stream_handle: ControlStreamHandle, pub(super) actor_manager: Arc, @@ -272,9 +270,6 @@ pub(super) struct LocalBarrierWorker { barrier_event_rx: UnboundedReceiver, actor_failure_rx: UnboundedReceiver<(ActorId, StreamError)>, - - /// Cached result of [`Self::try_find_root_failure`]. - cached_root_failure: Option, } impl LocalBarrierWorker { @@ -289,14 +284,12 @@ impl LocalBarrierWorker { }, )); Self { - failure_actors: HashMap::default(), state: ManagedBarrierState::new(actor_manager.clone(), shared_context.clone()), control_stream_handle: ControlStreamHandle::empty(), actor_manager, current_shared_context: shared_context, barrier_event_rx: event_rx, actor_failure_rx: failure_rx, - cached_root_failure: None, } } @@ -543,19 +536,6 @@ impl LocalBarrierWorker { request.actor_ids_to_collect ); - for actor_id in &request.actor_ids_to_collect { - if self.failure_actors.contains_key(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(StreamError::barrier_send( - barrier.clone(), - *actor_id, - "actor has already failed", - )); - } - } - self.state.transform_to_issued(barrier, request)?; Ok(()) } @@ -596,8 +576,7 @@ impl LocalBarrierWorker { err: StreamError, err_context: &'static str, ) { - self.add_failure(actor_id, err.clone()); - let root_err = self.try_find_root_failure().await.unwrap(); // always `Some` because we just added one + let root_err = self.try_find_root_failure(err).await; if let Some(actor_state) = self.state.actor_states.get(&actor_id) && (!actor_state.inflight_barriers.is_empty() || actor_state.is_running()) @@ -616,10 +595,7 @@ impl LocalBarrierWorker { /// This is similar to [`Self::notify_actor_failure`], but since there's not always an actor failure, /// the given `err` will be used if there's no root failure found. async fn notify_other_failure(&mut self, err: StreamError, message: impl Into) { - let root_err = self - .try_find_root_failure() - .await - .unwrap_or_else(|| ScoredStreamError::new(err)); + let root_err = self.try_find_root_failure(err).await; self.control_stream_handle.reset_stream_with_err( anyhow!(root_err) @@ -628,40 +604,24 @@ impl LocalBarrierWorker { ); } - fn add_failure(&mut self, actor_id: ActorId, err: StreamError) { - if let Some(prev_err) = self.failure_actors.insert(actor_id, err) { - warn!( - actor_id, - prev_err = %prev_err.as_report(), - "actor error overwritten" - ); - } - } - /// Collect actor errors for a while and find the one that might be the root cause. /// /// Returns `None` if there's no actor error received. - async fn try_find_root_failure(&mut self) -> Option { - if self.cached_root_failure.is_some() { - return self.cached_root_failure.clone(); - } - + async fn try_find_root_failure(&mut self, first_err: StreamError) -> ScoredStreamError { + let mut later_errs = vec![]; // fetch more actor errors within a timeout let _ = tokio::time::timeout(Duration::from_secs(3), async { - while let Some((actor_id, error)) = self.actor_failure_rx.recv().await { - self.add_failure(actor_id, error); + while let Some((_, error)) = self.actor_failure_rx.recv().await { + later_errs.push(error); } }) .await; - // Find the error with highest score. - self.cached_root_failure = self - .failure_actors - .values() + once(first_err) + .chain(later_errs.into_iter()) .map(|e| ScoredStreamError::new(e.clone())) - .max_by_key(|e| e.score); - - self.cached_root_failure.clone() + .max_by_key(|e| e.score) + .expect("non-empty") } } From c626c59da464b98f3d498030060741dcde2497c9 Mon Sep 17 00:00:00 2001 From: Dylan Date: Wed, 13 Nov 2024 16:14:02 +0800 Subject: [PATCH 06/16] chore(iceberg): suppress java S3FileIO warning (#19371) --- src/connector/src/connector_common/iceberg/mod.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/connector/src/connector_common/iceberg/mod.rs b/src/connector/src/connector_common/iceberg/mod.rs index 77a18368b0000..3d1b016577c6d 100644 --- a/src/connector/src/connector_common/iceberg/mod.rs +++ b/src/connector/src/connector_common/iceberg/mod.rs @@ -185,6 +185,10 @@ impl IcebergCommon { "org.apache.iceberg.aws.s3.S3FileIO".to_string(), ); + // suppress log of S3FileIO like: Unclosed S3FileIO instance created by... + java_catalog_configs + .insert("init-creation-stacktrace".to_string(), "false".to_string()); + if let Some(endpoint) = &self.endpoint { java_catalog_configs .insert("s3.endpoint".to_string(), endpoint.clone().to_string()); From 1aed314c2d8944da22b6d9c3c4d07c7683c2a98b Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 13 Nov 2024 16:31:16 +0800 Subject: [PATCH 07/16] feat(storage): support clear the shared buffer of a subset of tables (#19293) --- .../event_handler/hummock_event_handler.rs | 384 ++++++++++++++++-- src/storage/src/hummock/event_handler/mod.rs | 6 +- .../src/hummock/event_handler/uploader/mod.rs | 122 +++++- .../hummock/event_handler/uploader/spiller.rs | 16 +- .../event_handler/uploader/task_manager.rs | 11 +- .../event_handler/uploader/test_utils.rs | 85 ++-- .../src/hummock/store/hummock_storage.rs | 12 +- 7 files changed, 530 insertions(+), 106 deletions(-) diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index 3e3acf4b48ef9..0b8cadaf4c97e 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -88,6 +88,16 @@ impl BufferTracker { ) } + #[cfg(test)] + fn for_test_with_config(flush_threshold: usize, min_batch_flush_size: usize) -> Self { + Self::new( + usize::MAX, + flush_threshold, + GenericGauge::new("test", "test").unwrap(), + min_batch_flush_size, + ) + } + fn new( capacity: usize, flush_threshold: usize, @@ -232,12 +242,24 @@ impl HummockEventHandler { let upload_compactor_context = compactor_context.clone(); let upload_task_latency = state_store_metrics.uploader_upload_task_latency.clone(); let wait_poll_latency = state_store_metrics.uploader_wait_poll_latency.clone(); + let recent_versions = RecentVersions::new( + pinned_version, + compactor_context + .storage_opts + .max_cached_recent_versions_number, + state_store_metrics.clone(), + ); + let buffer_tracker = BufferTracker::from_storage_opts( + &compactor_context.storage_opts, + state_store_metrics.uploader_uploading_task_size.clone(), + ); Self::new_inner( version_update_rx, - pinned_version, compactor_context.sstable_store.clone(), state_store_metrics, - &compactor_context.storage_opts, + CacheRefillConfig::from_storage_opts(&compactor_context.storage_opts), + recent_versions, + buffer_tracker, Arc::new(move |payload, task_info| { static NEXT_UPLOAD_TASK_ID: LazyLock = LazyLock::new(|| AtomicUsize::new(0)); @@ -288,22 +310,20 @@ impl HummockEventHandler { fn new_inner( version_update_rx: UnboundedReceiver, - pinned_version: PinnedVersion, sstable_store: SstableStoreRef, state_store_metrics: Arc, - storage_opts: &StorageOpts, + refill_config: CacheRefillConfig, + recent_versions: RecentVersions, + buffer_tracker: BufferTracker, spawn_upload_task: SpawnUploadTask, spawn_refill_task: SpawnRefillTask, ) -> Self { let (hummock_event_tx, hummock_event_rx) = event_channel(state_store_metrics.event_handler_pending_event.clone()); - let (version_update_notifier_tx, _) = tokio::sync::watch::channel(pinned_version.clone()); + let (version_update_notifier_tx, _) = + tokio::sync::watch::channel(recent_versions.latest_version().clone()); let version_update_notifier_tx = Arc::new(version_update_notifier_tx); let read_version_mapping = Arc::new(RwLock::new(HashMap::default())); - let buffer_tracker = BufferTracker::from_storage_opts( - storage_opts, - state_store_metrics.uploader_uploading_task_size.clone(), - ); let metrics = HummockEventHandlerMetrics { event_handler_on_upload_finish_latency: state_store_metrics @@ -319,27 +339,18 @@ impl HummockEventHandler { let uploader = HummockUploader::new( state_store_metrics.clone(), - pinned_version.clone(), + recent_versions.latest_version().clone(), spawn_upload_task, buffer_tracker, - storage_opts, - ); - let refiller = CacheRefiller::new( - CacheRefillConfig::from_storage_opts(storage_opts), - sstable_store, - spawn_refill_task, ); + let refiller = CacheRefiller::new(refill_config, sstable_store, spawn_refill_task); Self { hummock_event_tx, hummock_event_rx, version_update_rx, version_update_notifier_tx, - recent_versions: Arc::new(ArcSwap::from_pointee(RecentVersions::new( - pinned_version, - storage_opts.max_cached_recent_versions_number, - state_store_metrics, - ))), + recent_versions: Arc::new(ArcSwap::from_pointee(recent_versions)), read_version_mapping, local_read_version_mapping: Default::default(), uploader, @@ -465,22 +476,24 @@ impl HummockEventHandler { .start_sync_epoch(new_sync_epoch, sync_result_sender, table_ids); } - fn handle_clear(&mut self, notifier: oneshot::Sender<()>) { + fn handle_clear(&mut self, notifier: oneshot::Sender<()>, table_ids: Option>) { info!( current_version_id = ?self.uploader.hummock_version().id(), "handle clear event" ); - self.uploader.clear(); + self.uploader.clear(table_ids.clone()); - assert!( - self.local_read_version_mapping.is_empty(), - "read version mapping not empty when clear. remaining tables: {:?}", - self.local_read_version_mapping - .values() - .map(|(_, read_version)| read_version.read().table_id()) - .collect_vec() - ); + if table_ids.is_none() { + assert!( + self.local_read_version_mapping.is_empty(), + "read version mapping not empty when clear. remaining tables: {:?}", + self.local_read_version_mapping + .values() + .map(|(_, read_version)| read_version.read().table_id()) + .collect_vec() + ); + } // Notify completion of the Clear event. let _ = notifier.send(()).inspect_err(|e| { @@ -634,8 +647,8 @@ impl HummockEventHandler { } => { self.handle_sync_epoch(new_sync_epoch, sync_result_sender, table_ids); } - HummockEvent::Clear(notifier) => { - self.handle_clear(notifier); + HummockEvent::Clear(notifier, table_ids) => { + self.handle_clear(notifier, table_ids); } HummockEvent::Shutdown => { unreachable!("shutdown is handled specially") @@ -837,6 +850,7 @@ mod tests { use futures::FutureExt; use parking_lot::Mutex; use risingwave_common::bitmap::Bitmap; + use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::{test_epoch, EpochExt}; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; @@ -846,15 +860,22 @@ mod tests { use tokio::sync::mpsc::unbounded_channel; use tokio::sync::oneshot; - use crate::hummock::event_handler::refiller::CacheRefiller; - use crate::hummock::event_handler::uploader::test_utils::{gen_imm, TEST_TABLE_ID}; + use crate::hummock::event_handler::hummock_event_handler::BufferTracker; + use crate::hummock::event_handler::refiller::{CacheRefillConfig, CacheRefiller}; + use crate::hummock::event_handler::uploader::test_utils::{ + gen_imm, gen_imm_inner, prepare_uploader_order_test_spawn_task_fn, TEST_TABLE_ID, + }; use crate::hummock::event_handler::uploader::UploadTaskOutput; - use crate::hummock::event_handler::{HummockEvent, HummockEventHandler}; + use crate::hummock::event_handler::{ + HummockEvent, HummockEventHandler, HummockReadVersionRef, LocalInstanceGuard, + }; use crate::hummock::iterator::test_utils::mock_sstable_store; use crate::hummock::local_version::pinned_version::PinnedVersion; + use crate::hummock::local_version::recent_versions::RecentVersions; use crate::hummock::store::version::{StagingData, VersionUpdate}; use crate::hummock::test_utils::default_opts_for_test; use crate::hummock::HummockError; + use crate::mem_table::ImmutableMemtable; use crate::monitor::HummockStateStoreMetrics; use crate::store::SealCurrentEpochOptions; @@ -884,12 +905,19 @@ mod tests { let (tx, rx) = oneshot::channel(); let rx = Arc::new(Mutex::new(Some(rx))); + let storage_opt = default_opts_for_test(); + let metrics = Arc::new(HummockStateStoreMetrics::unused()); + let event_handler = HummockEventHandler::new_inner( version_update_rx, - initial_version.clone(), mock_sstable_store().await, - Arc::new(HummockStateStoreMetrics::unused()), - &default_opts_for_test(), + metrics.clone(), + CacheRefillConfig::from_storage_opts(&storage_opt), + RecentVersions::new(initial_version.clone(), 10, metrics.clone()), + BufferTracker::from_storage_opts( + &storage_opt, + metrics.uploader_uploading_task_size.clone(), + ), Arc::new(move |_, info| { assert_eq!(info.epochs.len(), 1); let epoch = info.epochs[0]; @@ -918,7 +946,7 @@ mod tests { let send_event = |event| event_tx.send(event).unwrap(); - let _join_handle = spawn(event_handler.start_hummock_event_handler_worker()); + let join_handle = spawn(event_handler.start_hummock_event_handler_worker()); let (read_version, guard) = { let (tx, rx) = oneshot::channel(); @@ -1001,5 +1029,281 @@ mod tests { tx.send(()).unwrap(); rx1.await.unwrap().unwrap_err(); rx2.await.unwrap().unwrap_err(); + + send_event(HummockEvent::Shutdown); + join_handle.await.unwrap(); + } + + #[tokio::test] + async fn test_clear_tables() { + let table_id1 = TableId::new(1); + let table_id2 = TableId::new(2); + let epoch0 = test_epoch(233); + + let initial_version = PinnedVersion::new( + HummockVersion::from_rpc_protobuf(&PbHummockVersion { + id: 1, + state_table_info: HashMap::from_iter([ + ( + table_id1.table_id, + StateTableInfo { + committed_epoch: epoch0, + compaction_group_id: StaticCompactionGroupId::StateDefault as _, + }, + ), + ( + table_id2.table_id, + StateTableInfo { + committed_epoch: epoch0, + compaction_group_id: StaticCompactionGroupId::StateDefault as _, + }, + ), + ]), + ..Default::default() + }), + unbounded_channel().0, + ); + + let (_version_update_tx, version_update_rx) = unbounded_channel(); + + let epoch1 = epoch0.next_epoch(); + let epoch2 = epoch1.next_epoch(); + let epoch3 = epoch2.next_epoch(); + + let imm_size = gen_imm_inner(TEST_TABLE_ID, epoch1, 0, None).await.size(); + + // The buffer can hold at most 1 imm. When a new imm is added, the previous one will be spilled, and the newly added one will be retained. + let buffer_tracker = BufferTracker::for_test_with_config(imm_size * 2 - 1, 1); + let memory_limiter = buffer_tracker.get_memory_limiter().clone(); + + let gen_imm = |table_id, epoch, spill_offset| { + let imm = gen_imm_inner(table_id, epoch, spill_offset, Some(&*memory_limiter)) + .now_or_never() + .unwrap(); + assert_eq!(imm.size(), imm_size); + imm + }; + let imm1_1 = gen_imm(table_id1, epoch1, 0); + let imm1_2_1 = gen_imm(table_id1, epoch2, 0); + + let storage_opt = default_opts_for_test(); + let metrics = Arc::new(HummockStateStoreMetrics::unused()); + + let (spawn_task, new_task_notifier) = prepare_uploader_order_test_spawn_task_fn(false); + + let event_handler = HummockEventHandler::new_inner( + version_update_rx, + mock_sstable_store().await, + metrics.clone(), + CacheRefillConfig::from_storage_opts(&storage_opt), + RecentVersions::new(initial_version.clone(), 10, metrics.clone()), + buffer_tracker, + spawn_task, + CacheRefiller::default_spawn_refill_task(), + ); + + let event_tx = event_handler.event_sender(); + + let send_event = |event| event_tx.send(event).unwrap(); + let flush_event = || async { + let (tx, rx) = oneshot::channel(); + send_event(HummockEvent::FlushEvent(tx)); + rx.await.unwrap(); + }; + let start_epoch = |table_id, epoch| { + send_event(HummockEvent::StartEpoch { + epoch, + table_ids: HashSet::from_iter([table_id]), + }) + }; + let init_epoch = |instance: &LocalInstanceGuard, init_epoch| { + send_event(HummockEvent::InitEpoch { + instance_id: instance.instance_id, + init_epoch, + }) + }; + let write_imm = |read_version: &HummockReadVersionRef, + instance: &LocalInstanceGuard, + imm: &ImmutableMemtable| { + read_version + .write() + .update(VersionUpdate::Staging(StagingData::ImmMem(imm.clone()))); + + send_event(HummockEvent::ImmToUploader { + instance_id: instance.instance_id, + imm: imm.clone(), + }); + }; + let seal_epoch = |instance: &LocalInstanceGuard, next_epoch| { + send_event(HummockEvent::LocalSealEpoch { + instance_id: instance.instance_id, + next_epoch, + opts: SealCurrentEpochOptions::for_test(), + }) + }; + let sync_epoch = |table_id, new_sync_epoch| { + let (tx, rx) = oneshot::channel(); + send_event(HummockEvent::SyncEpoch { + new_sync_epoch, + sync_result_sender: tx, + table_ids: HashSet::from_iter([table_id]), + }); + rx + }; + + let join_handle = spawn(event_handler.start_hummock_event_handler_worker()); + + let (read_version1, guard1) = { + let (tx, rx) = oneshot::channel(); + send_event(HummockEvent::RegisterReadVersion { + table_id: table_id1, + new_read_version_sender: tx, + is_replicated: false, + vnodes: Arc::new(Bitmap::ones(VirtualNode::COUNT_FOR_TEST)), + }); + rx.await.unwrap() + }; + + let (read_version2, guard2) = { + let (tx, rx) = oneshot::channel(); + send_event(HummockEvent::RegisterReadVersion { + table_id: table_id2, + new_read_version_sender: tx, + is_replicated: false, + vnodes: Arc::new(Bitmap::ones(VirtualNode::COUNT_FOR_TEST)), + }); + rx.await.unwrap() + }; + + // prepare data of table1 + let (task1_1_finish_tx, task1_1_rx) = { + start_epoch(table_id1, epoch1); + + init_epoch(&guard1, epoch1); + + write_imm(&read_version1, &guard1, &imm1_1); + + start_epoch(table_id1, epoch2); + + seal_epoch(&guard1, epoch2); + + let (wait_task_start, task_finish_tx) = new_task_notifier(HashMap::from_iter([( + guard1.instance_id, + vec![imm1_1.batch_id()], + )])); + + let mut rx = sync_epoch(table_id1, epoch1); + wait_task_start.await; + assert!(poll_fn(|cx| Poll::Ready(rx.poll_unpin(cx).is_pending())).await); + + write_imm(&read_version1, &guard1, &imm1_2_1); + flush_event().await; + + (task_finish_tx, rx) + }; + // by now, the state in uploader of table_id1 + // unsync: epoch2 -> [imm1_2] + // syncing: epoch1 -> [imm1_1] + + let (task1_2_finish_tx, _finish_txs) = { + let mut finish_txs = vec![]; + let imm2_1_1 = gen_imm(table_id2, epoch1, 0); + start_epoch(table_id2, epoch1); + init_epoch(&guard2, epoch1); + let (wait_task_start, task1_2_finish_tx) = new_task_notifier(HashMap::from_iter([( + guard1.instance_id, + vec![imm1_2_1.batch_id()], + )])); + write_imm(&read_version2, &guard2, &imm2_1_1); + wait_task_start.await; + + let imm2_1_2 = gen_imm(table_id2, epoch1, 1); + let (wait_task_start, finish_tx) = new_task_notifier(HashMap::from_iter([( + guard2.instance_id, + vec![imm2_1_2.batch_id(), imm2_1_1.batch_id()], + )])); + finish_txs.push(finish_tx); + write_imm(&read_version2, &guard2, &imm2_1_2); + wait_task_start.await; + + let imm2_1_3 = gen_imm(table_id2, epoch1, 2); + write_imm(&read_version2, &guard2, &imm2_1_3); + start_epoch(table_id2, epoch2); + seal_epoch(&guard2, epoch2); + let (wait_task_start, finish_tx) = new_task_notifier(HashMap::from_iter([( + guard2.instance_id, + vec![imm2_1_3.batch_id()], + )])); + finish_txs.push(finish_tx); + let _sync_rx = sync_epoch(table_id2, epoch1); + wait_task_start.await; + + let imm2_2_1 = gen_imm(table_id2, epoch2, 0); + write_imm(&read_version2, &guard2, &imm2_2_1); + flush_event().await; + let imm2_2_2 = gen_imm(table_id2, epoch2, 1); + write_imm(&read_version2, &guard2, &imm2_2_2); + let (wait_task_start, finish_tx) = new_task_notifier(HashMap::from_iter([( + guard2.instance_id, + vec![imm2_2_2.batch_id(), imm2_2_1.batch_id()], + )])); + finish_txs.push(finish_tx); + wait_task_start.await; + + let imm2_2_3 = gen_imm(table_id2, epoch2, 2); + write_imm(&read_version2, &guard2, &imm2_2_3); + + // by now, the state in uploader of table_id2 + // syncing: epoch1 -> spill: [imm2_1_2, imm2_1_1], sync: [imm2_1_3] + // unsync: epoch2 -> spilling: [imm2_2_2, imm2_2_1], imm: [imm2_2_3] + // the state in uploader of table_id1 + // unsync: epoch2 -> spilling [imm1_2] + // syncing: epoch1 -> [imm1_1] + + drop(guard2); + let (clear_tx, clear_rx) = oneshot::channel(); + send_event(HummockEvent::Clear( + clear_tx, + Some(HashSet::from_iter([table_id2])), + )); + clear_rx.await.unwrap(); + (task1_2_finish_tx, finish_txs) + }; + + let imm1_2_2 = gen_imm(table_id1, epoch2, 1); + write_imm(&read_version1, &guard1, &imm1_2_2); + start_epoch(table_id1, epoch3); + seal_epoch(&guard1, epoch3); + + let (tx2, mut sync_rx2) = oneshot::channel(); + let (wait_task_start, task1_2_2_finish_tx) = new_task_notifier(HashMap::from_iter([( + guard1.instance_id, + vec![imm1_2_2.batch_id()], + )])); + send_event(HummockEvent::SyncEpoch { + new_sync_epoch: epoch2, + sync_result_sender: tx2, + table_ids: HashSet::from_iter([table_id1]), + }); + wait_task_start.await; + assert!(poll_fn(|cx| Poll::Ready(sync_rx2.poll_unpin(cx).is_pending())).await); + + task1_1_finish_tx.send(()).unwrap(); + let sync_data1 = task1_1_rx.await.unwrap().unwrap(); + sync_data1 + .uploaded_ssts + .iter() + .all(|sst| sst.epochs() == &vec![epoch1]); + task1_2_finish_tx.send(()).unwrap(); + assert!(poll_fn(|cx| Poll::Ready(sync_rx2.poll_unpin(cx).is_pending())).await); + task1_2_2_finish_tx.send(()).unwrap(); + let sync_data2 = sync_rx2.await.unwrap().unwrap(); + sync_data2 + .uploaded_ssts + .iter() + .all(|sst| sst.epochs() == &vec![epoch2]); + + send_event(HummockEvent::Shutdown); + join_handle.await.unwrap(); } } diff --git a/src/storage/src/hummock/event_handler/mod.rs b/src/storage/src/hummock/event_handler/mod.rs index 910c567e5d4da..46b44c051fdff 100644 --- a/src/storage/src/hummock/event_handler/mod.rs +++ b/src/storage/src/hummock/event_handler/mod.rs @@ -65,7 +65,7 @@ pub enum HummockEvent { }, /// Clear shared buffer and reset all states - Clear(oneshot::Sender<()>), + Clear(oneshot::Sender<()>, Option>), Shutdown, @@ -122,7 +122,9 @@ impl HummockEvent { table_ids, } => format!("AwaitSyncEpoch epoch {} {:?}", new_sync_epoch, table_ids), - HummockEvent::Clear(_) => "Clear".to_string(), + HummockEvent::Clear(_, table_ids) => { + format!("Clear {:?}", table_ids) + } HummockEvent::Shutdown => "Shutdown".to_string(), diff --git a/src/storage/src/hummock/event_handler/uploader/mod.rs b/src/storage/src/hummock/event_handler/uploader/mod.rs index 8326b3b876890..96b565c00ef49 100644 --- a/src/storage/src/hummock/event_handler/uploader/mod.rs +++ b/src/storage/src/hummock/event_handler/uploader/mod.rs @@ -53,7 +53,6 @@ use crate::hummock::store::version::StagingSstableInfo; use crate::hummock::{HummockError, HummockResult, ImmutableMemtable}; use crate::mem_table::ImmId; use crate::monitor::HummockStateStoreMetrics; -use crate::opts::StorageOpts; use crate::store::SealCurrentEpochOptions; /// Take epoch data inclusively before `epoch` out from `data` @@ -784,6 +783,7 @@ struct UnsyncData { // An index as a mapping from instance id to its table id instance_table_id: HashMap, unsync_epochs: HashMap>, + spilled_data: HashMap, HashSet)>, } impl UnsyncData { @@ -903,6 +903,43 @@ impl UnsyncData { None } } + + fn clear_tables(&mut self, table_ids: &HashSet, task_manager: &mut TaskManager) { + for table_id in table_ids { + if let Some(table_unsync_data) = self.table_data.remove(table_id) { + for task_id in table_unsync_data.spill_tasks.into_values().flatten() { + if let Some(task_status) = task_manager.abort_task(task_id) { + must_match!(task_status, UploadingTaskStatus::Spilling(spill_table_ids) => { + assert!(spill_table_ids.is_subset(table_ids)); + }); + } + if let Some((_, spill_table_ids)) = self.spilled_data.remove(&task_id) { + assert!(spill_table_ids.is_subset(table_ids)); + } + } + assert!( + table_unsync_data.instance_data.is_empty(), + "should be clear when dropping the read version instance" + ); + } + } + debug_assert!(self + .spilled_data + .values() + .all(|(_, spill_table_ids)| spill_table_ids.is_disjoint(table_ids))); + self.unsync_epochs.retain(|_, unsync_epoch_table_ids| { + if !unsync_epoch_table_ids.is_disjoint(table_ids) { + assert!(unsync_epoch_table_ids.is_subset(table_ids)); + false + } else { + true + } + }); + assert!(self + .instance_table_id + .values() + .all(|table_id| !table_ids.contains(table_id))); + } } impl UploaderData { @@ -958,7 +995,7 @@ impl UploaderData { ); } for task_id in task_ids { - if self.spilled_data.contains_key(&task_id) { + if self.unsync_data.spilled_data.contains_key(&task_id) { spilled_tasks.insert(task_id); } else { uploading_tasks.insert(task_id); @@ -988,8 +1025,11 @@ impl UploaderData { .iter() .rev() .map(|task_id| { - let (sst, spill_table_ids) = - self.spilled_data.remove(task_id).expect("should exist"); + let (sst, spill_table_ids) = self + .unsync_data + .spilled_data + .remove(task_id) + .expect("should exist"); assert!( spill_table_ids.is_subset(&table_ids), "spilled tabled ids {:?} not a subset of sync table id {:?}", @@ -1057,7 +1097,6 @@ impl UploaderContext { pinned_version: PinnedVersion, spawn_upload_task: SpawnUploadTask, buffer_tracker: BufferTracker, - _config: &StorageOpts, stats: Arc, ) -> Self { UploaderContext { @@ -1079,20 +1118,52 @@ struct UploaderData { syncing_data: BTreeMap, task_manager: TaskManager, - spilled_data: HashMap, HashSet)>, next_sync_id: usize, } impl UploaderData { fn abort(self, err: impl Fn() -> HummockError) { - self.task_manager.abort(); + self.task_manager.abort_all_tasks(); for syncing_data in self.syncing_data.into_values() { send_sync_result(syncing_data.sync_result_sender, Err(err())); } } + fn clear_tables(&mut self, table_ids: HashSet) { + if table_ids.is_empty() { + return; + } + self.unsync_data + .clear_tables(&table_ids, &mut self.task_manager); + self.syncing_data.retain(|sync_id, syncing_data| { + if !syncing_data.table_ids.is_disjoint(&table_ids) { + assert!(syncing_data.table_ids.is_subset(&table_ids)); + for task_id in &syncing_data.remaining_uploading_tasks { + match self + .task_manager + .abort_task(*task_id) + .expect("should exist") + { + UploadingTaskStatus::Spilling(spill_table_ids) => { + assert!(spill_table_ids.is_subset(&table_ids)); + } + UploadingTaskStatus::Sync(task_sync_id) => { + assert_eq!(sync_id, &task_sync_id); + } + } + } + false + } else { + true + } + }); + + self.check_upload_task_consistency(); + } + fn min_uncommitted_sst_id(&self) -> Option { - self.spilled_data + self.unsync_data + .spilled_data .values() .map(|(s, _)| s) .chain(self.syncing_data.values().flat_map(|s| s.uploaded.iter())) @@ -1141,7 +1212,6 @@ impl HummockUploader { pinned_version: PinnedVersion, spawn_upload_task: SpawnUploadTask, buffer_tracker: BufferTracker, - config: &StorageOpts, ) -> Self { Self { state: UploaderState::Working(UploaderData::default()), @@ -1149,7 +1219,6 @@ impl HummockUploader { pinned_version, spawn_upload_task, buffer_tracker, - config, state_store_metrics, ), } @@ -1308,15 +1377,21 @@ impl HummockUploader { } } - pub(crate) fn clear(&mut self) { - if let UploaderState::Working(data) = replace( - &mut self.state, - UploaderState::Working(UploaderData::default()), - ) { - data.abort(|| HummockError::other("uploader is reset")); - } + pub(crate) fn clear(&mut self, table_ids: Option>) { + if let Some(table_ids) = table_ids { + if let UploaderState::Working(data) = &mut self.state { + data.clear_tables(table_ids); + } + } else { + if let UploaderState::Working(data) = replace( + &mut self.state, + UploaderState::Working(UploaderData::default()), + ) { + data.abort(|| HummockError::other("uploader is reset")); + } - self.context.stats.uploader_syncing_epoch_count.set(0); + self.context.stats.uploader_syncing_epoch_count.set(0); + } } pub(crate) fn may_destroy_instance(&mut self, instance_id: LocalInstanceId) { @@ -1331,10 +1406,11 @@ impl HummockUploader { .into_values() .flat_map(|task_ids| task_ids.into_iter()) .filter(|task_id| { - if let Some((_, table_ids)) = data.spilled_data.get_mut(task_id) { + if let Some((_, table_ids)) = data.unsync_data.spilled_data.get_mut(task_id) + { assert!(table_ids.remove(&removed_table_data.table_id)); if table_ids.is_empty() { - data.spilled_data.remove(task_id); + data.unsync_data.spilled_data.remove(task_id); } false } else { @@ -1422,7 +1498,7 @@ impl UploaderData { .collect(); let mut spill_task_table_id_from_manager: HashMap<_, HashSet<_>> = HashMap::new(); - for (task_id, (_, table_ids)) in &self.spilled_data { + for (task_id, (_, table_ids)) in &self.unsync_data.spilled_data { spill_task_table_id_from_manager.insert(*task_id, table_ids.clone()); } let mut syncing_task_from_manager: HashMap<_, HashSet<_>> = HashMap::new(); @@ -1473,7 +1549,9 @@ impl HummockUploader { data.may_notify_sync_task(&self.context); } UploadingTaskStatus::Spilling(table_ids) => { - data.spilled_data.insert(task_id, (sst.clone(), table_ids)); + data.unsync_data + .spilled_data + .insert(task_id, (sst.clone(), table_ids)); } } data.check_upload_task_consistency(); diff --git a/src/storage/src/hummock/event_handler/uploader/spiller.rs b/src/storage/src/hummock/event_handler/uploader/spiller.rs index ba04d85856ace..4e560c36eacf0 100644 --- a/src/storage/src/hummock/event_handler/uploader/spiller.rs +++ b/src/storage/src/hummock/event_handler/uploader/spiller.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cmp::Ordering; use std::collections::{HashMap, HashSet}; use risingwave_common::catalog::TableId; @@ -75,7 +76,20 @@ impl<'a> Spiller<'a> { if let Some(unsync_epoch_id) = self .epoch_info .iter() - .max_by_key(|(_, info)| info.payload_size) + .max_by( + |(UnsyncEpochId(_, table1), info1), (UnsyncEpochId(_, table2), info2)| { + info1.payload_size.cmp(&info2.payload_size).then_with(|| { + if !cfg!(test) { + Ordering::Equal + } else { + assert_ne!(table1, table2); + // enforce deterministic spill order in test + // smaller table id will be spilled first. + table2.cmp(table1) + } + }) + }, + ) .map(|(unsync_epoch_id, _)| *unsync_epoch_id) { let spill_epoch = unsync_epoch_id.epoch(); diff --git a/src/storage/src/hummock/event_handler/uploader/task_manager.rs b/src/storage/src/hummock/event_handler/uploader/task_manager.rs index 2347be1ed57eb..fd53fae1db322 100644 --- a/src/storage/src/hummock/event_handler/uploader/task_manager.rs +++ b/src/storage/src/hummock/event_handler/uploader/task_manager.rs @@ -97,12 +97,21 @@ impl TaskManager { } } - pub(super) fn abort(self) { + pub(super) fn abort_all_tasks(self) { for task in self.tasks.into_values() { task.task.join_handle.abort(); } } + pub(super) fn abort_task(&mut self, task_id: UploadingTaskId) -> Option { + self.tasks.remove(&task_id).map(|entry| { + entry.task.join_handle.abort(); + self.task_order + .retain(|inflight_task_id| *inflight_task_id != task_id); + entry.status + }) + } + pub(super) fn spill( &mut self, context: &UploaderContext, diff --git a/src/storage/src/hummock/event_handler/uploader/test_utils.rs b/src/storage/src/hummock/event_handler/uploader/test_utils.rs index 6eb41bda52071..3e7b92624109a 100644 --- a/src/storage/src/hummock/event_handler/uploader/test_utils.rs +++ b/src/storage/src/hummock/event_handler/uploader/test_utils.rs @@ -45,8 +45,9 @@ use tokio::task::yield_now; use crate::hummock::event_handler::hummock_event_handler::BufferTracker; use crate::hummock::event_handler::uploader::uploader_imm::UploaderImm; use crate::hummock::event_handler::uploader::{ - HummockUploader, TableUnsyncData, UploadTaskInfo, UploadTaskOutput, UploadTaskPayload, - UploaderContext, UploaderData, UploaderState, UploadingTask, UploadingTaskId, + HummockUploader, SpawnUploadTask, TableUnsyncData, UploadTaskInfo, UploadTaskOutput, + UploadTaskPayload, UploaderContext, UploaderData, UploaderState, UploadingTask, + UploadingTaskId, }; use crate::hummock::event_handler::{LocalInstanceId, TEST_LOCAL_INSTANCE_ID}; use crate::hummock::local_version::pinned_version::PinnedVersion; @@ -121,7 +122,7 @@ pub(super) async fn gen_imm_with_limiter( gen_imm_inner(TEST_TABLE_ID, epoch, 0, limiter).await } -pub(super) async fn gen_imm_inner( +pub(crate) async fn gen_imm_inner( table_id: TableId, epoch: HummockEpoch, spill_offset: u16, @@ -176,12 +177,10 @@ where Fut: UploadOutputFuture, F: UploadFn, { - let config = StorageOpts::default(); UploaderContext::new( initial_pinned_version(), Arc::new(move |payload, task_info| spawn(upload_fn(payload, task_info))), BufferTracker::for_test(), - &config, Arc::new(HummockStateStoreMetrics::unused()), ) } @@ -191,15 +190,11 @@ where Fut: UploadOutputFuture, F: UploadFn, { - let config = StorageOpts { - ..Default::default() - }; HummockUploader::new( Arc::new(HummockStateStoreMetrics::unused()), initial_pinned_version(), Arc::new(move |payload, task_info| spawn(upload_fn(payload, task_info))), BufferTracker::for_test(), - &config, ) } @@ -272,16 +267,12 @@ impl HummockUploader { } #[expect(clippy::type_complexity)] -pub(crate) fn prepare_uploader_order_test( - config: &StorageOpts, +pub(crate) fn prepare_uploader_order_test_spawn_task_fn( skip_schedule: bool, ) -> ( - BufferTracker, - HummockUploader, + SpawnUploadTask, impl Fn(HashMap>) -> (BoxFuture<'static, ()>, oneshot::Sender<()>), ) { - let gauge = GenericGauge::new("test", "test").unwrap(); - let buffer_tracker = BufferTracker::from_storage_opts(config, gauge); // (the started task send the imm ids of payload, the started task wait for finish notify) #[allow(clippy::type_complexity)] let task_notifier_holder: Arc< @@ -304,35 +295,51 @@ pub(crate) fn prepare_uploader_order_test( (await_start_future, finish_tx) } }; + let spawn_fn = Arc::new({ + move |_, task_info: UploadTaskInfo| { + let task_notifier_holder = task_notifier_holder.clone(); + let task_item = task_notifier_holder.lock().pop_back(); + let start_epoch = *task_info.epochs.last().unwrap(); + let end_epoch = *task_info.epochs.first().unwrap(); + assert!(end_epoch >= start_epoch); + spawn(async move { + let ssts = gen_sstable_info(start_epoch, end_epoch); + if !skip_schedule { + let (start_tx, finish_rx) = task_item.unwrap(); + start_tx.send(task_info).unwrap(); + finish_rx + .await + .map_err(|_| HummockError::other("failed to receive rx"))?; + } + Ok(UploadTaskOutput { + new_value_ssts: ssts, + old_value_ssts: vec![], + wait_poll_timer: None, + }) + }) + } + }); + (spawn_fn, new_task_notifier) +} + +#[expect(clippy::type_complexity)] +pub(crate) fn prepare_uploader_order_test( + config: &StorageOpts, + skip_schedule: bool, +) -> ( + BufferTracker, + HummockUploader, + impl Fn(HashMap>) -> (BoxFuture<'static, ()>, oneshot::Sender<()>), +) { + let (spawn_fn, new_task_notifier) = prepare_uploader_order_test_spawn_task_fn(skip_schedule); + let gauge = GenericGauge::new("test", "test").unwrap(); + let buffer_tracker = BufferTracker::from_storage_opts(config, gauge); - let config = StorageOpts::default(); let uploader = HummockUploader::new( Arc::new(HummockStateStoreMetrics::unused()), initial_pinned_version(), - Arc::new({ - move |_, task_info: UploadTaskInfo| { - let task_notifier_holder = task_notifier_holder.clone(); - let task_item = task_notifier_holder.lock().pop_back(); - let start_epoch = *task_info.epochs.last().unwrap(); - let end_epoch = *task_info.epochs.first().unwrap(); - assert!(end_epoch >= start_epoch); - spawn(async move { - let ssts = gen_sstable_info(start_epoch, end_epoch); - if !skip_schedule { - let (start_tx, finish_rx) = task_item.unwrap(); - start_tx.send(task_info).unwrap(); - finish_rx.await.unwrap(); - } - Ok(UploadTaskOutput { - new_value_ssts: ssts, - old_value_ssts: vec![], - wait_poll_timer: None, - }) - }) - } - }), + spawn_fn, buffer_tracker.clone(), - &config, ); (buffer_tracker, uploader, new_task_notifier) } diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 3b4a143d0ad53..43bb08c44a0ba 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -513,11 +513,21 @@ impl HummockStorage { pub async fn clear_shared_buffer(&self) { let (tx, rx) = oneshot::channel(); self.hummock_event_sender - .send(HummockEvent::Clear(tx)) + .send(HummockEvent::Clear(tx, None)) .expect("should send success"); rx.await.expect("should wait success"); } + pub async fn clear_tables(&self, table_ids: HashSet) { + if !table_ids.is_empty() { + let (tx, rx) = oneshot::channel(); + self.hummock_event_sender + .send(HummockEvent::Clear(tx, Some(table_ids))) + .expect("should send success"); + rx.await.expect("should wait success"); + } + } + /// Declare the start of an epoch. This information is provided for spill so that the spill task won't /// include data of two or more syncs. // TODO: remove this method when we support spill task that can include data of more two or more syncs From a0ff192290dd767d9f8677709c3b218a4fad2773 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 14 Nov 2024 03:35:31 +0000 Subject: [PATCH 08/16] chore(deps): Bump cookie and express in /dashboard (#19324) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: xxchan --- dashboard/package-lock.json | 223 +++++++++++------------------------- dashboard/package.json | 2 +- 2 files changed, 71 insertions(+), 154 deletions(-) diff --git a/dashboard/package-lock.json b/dashboard/package-lock.json index ce861243e27cf..13322f53cd404 100644 --- a/dashboard/package-lock.json +++ b/dashboard/package-lock.json @@ -56,7 +56,7 @@ "eslint-plugin-n": "^15.2.5", "eslint-plugin-promise": "^6.0.1", "eslint-plugin-react": "^7.31.6", - "express": "^4.20.0", + "express": "^4.21.1", "prettier": "^2.7.1", "prettier-plugin-organize-imports": "^3.1.1", "typescript": "5.4.2" @@ -3850,21 +3850,6 @@ "integrity": "sha512-Tpp60P6IUJDTuOq/5Z8cdskzJujfwqfOTkrwIwj7IRISpnkJnT6SyJ4PCPnGMoFjC9ddhal5KVIYtAt97ix05A==", "dev": true }, - "node_modules/body-parser/node_modules/qs": { - "version": "6.13.0", - "resolved": "https://registry.npmjs.org/qs/-/qs-6.13.0.tgz", - "integrity": "sha512-+38qI9SOr8tfZ4QmJNplMUxqjbe7LKvvZgWdExBOmd+egZTtjLB67Gu0HRX3u/XOq7UU2Nx6nsjvS16Z9uwfpg==", - "dev": true, - "dependencies": { - "side-channel": "^1.0.6" - }, - "engines": { - "node": ">=0.6" - }, - "funding": { - "url": "https://github.com/sponsors/ljharb" - } - }, "node_modules/bootstrap-icons": { "version": "1.9.1", "resolved": "https://registry.npmjs.org/bootstrap-icons/-/bootstrap-icons-1.9.1.tgz", @@ -4275,9 +4260,9 @@ } }, "node_modules/cookie": { - "version": "0.6.0", - "resolved": "https://registry.npmjs.org/cookie/-/cookie-0.6.0.tgz", - "integrity": "sha512-U71cyTamuh1CRNCfpGY6to28lxvNwPG4Guz/EVjgf3Jmzv0vlDp1atT9eS5dDjMYHucpHbWns6Lwf3BKz6svdw==", + "version": "0.7.1", + "resolved": "https://registry.npmjs.org/cookie/-/cookie-0.7.1.tgz", + "integrity": "sha512-6DnInpx7SJ2AK3+CTUE/ZM0vWTUboZCegxhC2xiIydHR9jNuTAASBrfEpHhiGOZw/nX51bHt6YQl8jsGo4y/0w==", "dev": true, "engines": { "node": ">= 0.6" @@ -6177,9 +6162,9 @@ } }, "node_modules/express": { - "version": "4.20.0", - "resolved": "https://registry.npmjs.org/express/-/express-4.20.0.tgz", - "integrity": "sha512-pLdae7I6QqShF5PnNTCVn4hI91Dx0Grkn2+IAsMTgMIKuQVte2dN9PeGSSAME2FR8anOhVA62QDIUaWVfEXVLw==", + "version": "4.21.1", + "resolved": "https://registry.npmjs.org/express/-/express-4.21.1.tgz", + "integrity": "sha512-YSFlK1Ee0/GC8QaO91tHcDxJiE/X4FbpAyQWkxAvG6AXCuR65YzK8ua6D9hvi/TzUfZMpc+BwuM1IPw8fmQBiQ==", "dev": true, "dependencies": { "accepts": "~1.3.8", @@ -6187,14 +6172,14 @@ "body-parser": "1.20.3", "content-disposition": "0.5.4", "content-type": "~1.0.4", - "cookie": "0.6.0", + "cookie": "0.7.1", "cookie-signature": "1.0.6", "debug": "2.6.9", "depd": "2.0.0", "encodeurl": "~2.0.0", "escape-html": "~1.0.3", "etag": "~1.8.1", - "finalhandler": "1.2.0", + "finalhandler": "1.3.1", "fresh": "0.5.2", "http-errors": "2.0.0", "merge-descriptors": "1.0.3", @@ -6203,11 +6188,11 @@ "parseurl": "~1.3.3", "path-to-regexp": "0.1.10", "proxy-addr": "~2.0.7", - "qs": "6.11.0", + "qs": "6.13.0", "range-parser": "~1.2.1", "safe-buffer": "5.2.1", "send": "0.19.0", - "serve-static": "1.16.0", + "serve-static": "1.16.2", "setprototypeof": "1.2.0", "statuses": "2.0.1", "type-is": "~1.6.18", @@ -6403,13 +6388,13 @@ } }, "node_modules/finalhandler": { - "version": "1.2.0", - "resolved": "https://registry.npmjs.org/finalhandler/-/finalhandler-1.2.0.tgz", - "integrity": "sha512-5uXcUVftlQMFnWC9qu/svkWv3GTd2PfUhK/3PLkYNAe7FbqJMt3515HaxE6eRL74GdsriiwujiawdaB1BpEISg==", + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/finalhandler/-/finalhandler-1.3.1.tgz", + "integrity": "sha512-6BN9trH7bp3qvnrRyzsBz+g3lZxTNZTbVO2EV1CS0WIcDbawYVdYvGflME/9QP0h0pYlCDBCTjYa9nZzMDpyxQ==", "dev": true, "dependencies": { "debug": "2.6.9", - "encodeurl": "~1.0.2", + "encodeurl": "~2.0.0", "escape-html": "~1.0.3", "on-finished": "2.4.1", "parseurl": "~1.3.3", @@ -6429,6 +6414,15 @@ "ms": "2.0.0" } }, + "node_modules/finalhandler/node_modules/encodeurl": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/encodeurl/-/encodeurl-2.0.0.tgz", + "integrity": "sha512-Q0n9HRi4m6JuGIV1eFlmvJB7ZEVxu93IrMyiMsGC0lrMJMWzRgx6WGquyfQgZVb31vhGgXnfmPNNXmxnOkRBrg==", + "dev": true, + "engines": { + "node": ">= 0.8" + } + }, "node_modules/finalhandler/node_modules/ms": { "version": "2.0.0", "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", @@ -9388,12 +9382,12 @@ "integrity": "sha512-QFADYnsVoBMw1srW7OVKEYjG+MbIa49s54w1MA1EDY6r2r/sTcKKYqRX1f4GYvnXP7eN/Pe9HFcX+hwzmrXRHA==" }, "node_modules/qs": { - "version": "6.11.0", - "resolved": "https://registry.npmjs.org/qs/-/qs-6.11.0.tgz", - "integrity": "sha512-MvjoMCJwEarSbUYk5O+nmoSzSutSsTwF85zcHPQ9OrlFoZOYIjaqBAJIqIXjptyD5vThxGq52Xu/MaJzRkIk4Q==", + "version": "6.13.0", + "resolved": "https://registry.npmjs.org/qs/-/qs-6.13.0.tgz", + "integrity": "sha512-+38qI9SOr8tfZ4QmJNplMUxqjbe7LKvvZgWdExBOmd+egZTtjLB67Gu0HRX3u/XOq7UU2Nx6nsjvS16Z9uwfpg==", "dev": true, "dependencies": { - "side-channel": "^1.0.4" + "side-channel": "^1.0.6" }, "engines": { "node": ">=0.6" @@ -10206,63 +10200,27 @@ "dev": true }, "node_modules/serve-static": { - "version": "1.16.0", - "resolved": "https://registry.npmjs.org/serve-static/-/serve-static-1.16.0.tgz", - "integrity": "sha512-pDLK8zwl2eKaYrs8mrPZBJua4hMplRWJ1tIFksVC3FtBEBnl8dxgeHtsaMS8DhS9i4fLObaon6ABoc4/hQGdPA==", + "version": "1.16.2", + "resolved": "https://registry.npmjs.org/serve-static/-/serve-static-1.16.2.tgz", + "integrity": "sha512-VqpjJZKadQB/PEbEwvFdO43Ax5dFBZ2UECszz8bQ7pi7wt//PWe1P6MN7eCnjsatYtBT6EuiClbjSWP2WrIoTw==", "dev": true, "dependencies": { - "encodeurl": "~1.0.2", + "encodeurl": "~2.0.0", "escape-html": "~1.0.3", "parseurl": "~1.3.3", - "send": "0.18.0" + "send": "0.19.0" }, "engines": { "node": ">= 0.8.0" } }, - "node_modules/serve-static/node_modules/debug": { - "version": "2.6.9", - "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", - "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", - "dev": true, - "dependencies": { - "ms": "2.0.0" - } - }, - "node_modules/serve-static/node_modules/debug/node_modules/ms": { + "node_modules/serve-static/node_modules/encodeurl": { "version": "2.0.0", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", - "integrity": "sha512-Tpp60P6IUJDTuOq/5Z8cdskzJujfwqfOTkrwIwj7IRISpnkJnT6SyJ4PCPnGMoFjC9ddhal5KVIYtAt97ix05A==", - "dev": true - }, - "node_modules/serve-static/node_modules/ms": { - "version": "2.1.3", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", - "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==", - "dev": true - }, - "node_modules/serve-static/node_modules/send": { - "version": "0.18.0", - "resolved": "https://registry.npmjs.org/send/-/send-0.18.0.tgz", - "integrity": "sha512-qqWzuOjSFOuqPjFe4NOsMLafToQQwBSOEpS+FwEt3A2V3vKubTquT3vmLTQpFgMXp8AlFWFuP1qKaJZOtPpVXg==", + "resolved": "https://registry.npmjs.org/encodeurl/-/encodeurl-2.0.0.tgz", + "integrity": "sha512-Q0n9HRi4m6JuGIV1eFlmvJB7ZEVxu93IrMyiMsGC0lrMJMWzRgx6WGquyfQgZVb31vhGgXnfmPNNXmxnOkRBrg==", "dev": true, - "dependencies": { - "debug": "2.6.9", - "depd": "2.0.0", - "destroy": "1.2.0", - "encodeurl": "~1.0.2", - "escape-html": "~1.0.3", - "etag": "~1.8.1", - "fresh": "0.5.2", - "http-errors": "2.0.0", - "mime": "1.6.0", - "ms": "2.1.3", - "on-finished": "2.4.1", - "range-parser": "~1.2.1", - "statuses": "2.0.1" - }, "engines": { - "node": ">= 0.8.0" + "node": ">= 0.8" } }, "node_modules/set-blocking": { @@ -14629,15 +14587,6 @@ "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", "integrity": "sha512-Tpp60P6IUJDTuOq/5Z8cdskzJujfwqfOTkrwIwj7IRISpnkJnT6SyJ4PCPnGMoFjC9ddhal5KVIYtAt97ix05A==", "dev": true - }, - "qs": { - "version": "6.13.0", - "resolved": "https://registry.npmjs.org/qs/-/qs-6.13.0.tgz", - "integrity": "sha512-+38qI9SOr8tfZ4QmJNplMUxqjbe7LKvvZgWdExBOmd+egZTtjLB67Gu0HRX3u/XOq7UU2Nx6nsjvS16Z9uwfpg==", - "dev": true, - "requires": { - "side-channel": "^1.0.6" - } } } }, @@ -14930,9 +14879,9 @@ } }, "cookie": { - "version": "0.6.0", - "resolved": "https://registry.npmjs.org/cookie/-/cookie-0.6.0.tgz", - "integrity": "sha512-U71cyTamuh1CRNCfpGY6to28lxvNwPG4Guz/EVjgf3Jmzv0vlDp1atT9eS5dDjMYHucpHbWns6Lwf3BKz6svdw==", + "version": "0.7.1", + "resolved": "https://registry.npmjs.org/cookie/-/cookie-0.7.1.tgz", + "integrity": "sha512-6DnInpx7SJ2AK3+CTUE/ZM0vWTUboZCegxhC2xiIydHR9jNuTAASBrfEpHhiGOZw/nX51bHt6YQl8jsGo4y/0w==", "dev": true }, "cookie-signature": { @@ -16346,9 +16295,9 @@ } }, "express": { - "version": "4.20.0", - "resolved": "https://registry.npmjs.org/express/-/express-4.20.0.tgz", - "integrity": "sha512-pLdae7I6QqShF5PnNTCVn4hI91Dx0Grkn2+IAsMTgMIKuQVte2dN9PeGSSAME2FR8anOhVA62QDIUaWVfEXVLw==", + "version": "4.21.1", + "resolved": "https://registry.npmjs.org/express/-/express-4.21.1.tgz", + "integrity": "sha512-YSFlK1Ee0/GC8QaO91tHcDxJiE/X4FbpAyQWkxAvG6AXCuR65YzK8ua6D9hvi/TzUfZMpc+BwuM1IPw8fmQBiQ==", "dev": true, "requires": { "accepts": "~1.3.8", @@ -16356,14 +16305,14 @@ "body-parser": "1.20.3", "content-disposition": "0.5.4", "content-type": "~1.0.4", - "cookie": "0.6.0", + "cookie": "0.7.1", "cookie-signature": "1.0.6", "debug": "2.6.9", "depd": "2.0.0", "encodeurl": "~2.0.0", "escape-html": "~1.0.3", "etag": "~1.8.1", - "finalhandler": "1.2.0", + "finalhandler": "1.3.1", "fresh": "0.5.2", "http-errors": "2.0.0", "merge-descriptors": "1.0.3", @@ -16372,11 +16321,11 @@ "parseurl": "~1.3.3", "path-to-regexp": "0.1.10", "proxy-addr": "~2.0.7", - "qs": "6.11.0", + "qs": "6.13.0", "range-parser": "~1.2.1", "safe-buffer": "5.2.1", "send": "0.19.0", - "serve-static": "1.16.0", + "serve-static": "1.16.2", "setprototypeof": "1.2.0", "statuses": "2.0.1", "type-is": "~1.6.18", @@ -16537,13 +16486,13 @@ } }, "finalhandler": { - "version": "1.2.0", - "resolved": "https://registry.npmjs.org/finalhandler/-/finalhandler-1.2.0.tgz", - "integrity": "sha512-5uXcUVftlQMFnWC9qu/svkWv3GTd2PfUhK/3PLkYNAe7FbqJMt3515HaxE6eRL74GdsriiwujiawdaB1BpEISg==", + "version": "1.3.1", + "resolved": "https://registry.npmjs.org/finalhandler/-/finalhandler-1.3.1.tgz", + "integrity": "sha512-6BN9trH7bp3qvnrRyzsBz+g3lZxTNZTbVO2EV1CS0WIcDbawYVdYvGflME/9QP0h0pYlCDBCTjYa9nZzMDpyxQ==", "dev": true, "requires": { "debug": "2.6.9", - "encodeurl": "~1.0.2", + "encodeurl": "~2.0.0", "escape-html": "~1.0.3", "on-finished": "2.4.1", "parseurl": "~1.3.3", @@ -16560,6 +16509,12 @@ "ms": "2.0.0" } }, + "encodeurl": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/encodeurl/-/encodeurl-2.0.0.tgz", + "integrity": "sha512-Q0n9HRi4m6JuGIV1eFlmvJB7ZEVxu93IrMyiMsGC0lrMJMWzRgx6WGquyfQgZVb31vhGgXnfmPNNXmxnOkRBrg==", + "dev": true + }, "ms": { "version": "2.0.0", "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", @@ -18686,12 +18641,12 @@ "integrity": "sha512-QFADYnsVoBMw1srW7OVKEYjG+MbIa49s54w1MA1EDY6r2r/sTcKKYqRX1f4GYvnXP7eN/Pe9HFcX+hwzmrXRHA==" }, "qs": { - "version": "6.11.0", - "resolved": "https://registry.npmjs.org/qs/-/qs-6.11.0.tgz", - "integrity": "sha512-MvjoMCJwEarSbUYk5O+nmoSzSutSsTwF85zcHPQ9OrlFoZOYIjaqBAJIqIXjptyD5vThxGq52Xu/MaJzRkIk4Q==", + "version": "6.13.0", + "resolved": "https://registry.npmjs.org/qs/-/qs-6.13.0.tgz", + "integrity": "sha512-+38qI9SOr8tfZ4QmJNplMUxqjbe7LKvvZgWdExBOmd+egZTtjLB67Gu0HRX3u/XOq7UU2Nx6nsjvS16Z9uwfpg==", "dev": true, "requires": { - "side-channel": "^1.0.4" + "side-channel": "^1.0.6" } }, "quadprog": { @@ -19270,60 +19225,22 @@ } }, "serve-static": { - "version": "1.16.0", - "resolved": "https://registry.npmjs.org/serve-static/-/serve-static-1.16.0.tgz", - "integrity": "sha512-pDLK8zwl2eKaYrs8mrPZBJua4hMplRWJ1tIFksVC3FtBEBnl8dxgeHtsaMS8DhS9i4fLObaon6ABoc4/hQGdPA==", + "version": "1.16.2", + "resolved": "https://registry.npmjs.org/serve-static/-/serve-static-1.16.2.tgz", + "integrity": "sha512-VqpjJZKadQB/PEbEwvFdO43Ax5dFBZ2UECszz8bQ7pi7wt//PWe1P6MN7eCnjsatYtBT6EuiClbjSWP2WrIoTw==", "dev": true, "requires": { - "encodeurl": "~1.0.2", + "encodeurl": "~2.0.0", "escape-html": "~1.0.3", "parseurl": "~1.3.3", - "send": "0.18.0" + "send": "0.19.0" }, "dependencies": { - "debug": { - "version": "2.6.9", - "resolved": "https://registry.npmjs.org/debug/-/debug-2.6.9.tgz", - "integrity": "sha512-bC7ElrdJaJnPbAP+1EotYvqZsb3ecl5wi6Bfi6BJTUcNowp6cvspg0jXznRTKDjm/E7AdgFBVeAPVMNcKGsHMA==", - "dev": true, - "requires": { - "ms": "2.0.0" - }, - "dependencies": { - "ms": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.0.0.tgz", - "integrity": "sha512-Tpp60P6IUJDTuOq/5Z8cdskzJujfwqfOTkrwIwj7IRISpnkJnT6SyJ4PCPnGMoFjC9ddhal5KVIYtAt97ix05A==", - "dev": true - } - } - }, - "ms": { - "version": "2.1.3", - "resolved": "https://registry.npmjs.org/ms/-/ms-2.1.3.tgz", - "integrity": "sha512-6FlzubTLZG3J2a/NVCAleEhjzq5oxgHyaCU9yYXvcLsvoVaHJq/s5xXI6/XXP6tz7R9xAOtHnSO/tXtF3WRTlA==", + "encodeurl": { + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/encodeurl/-/encodeurl-2.0.0.tgz", + "integrity": "sha512-Q0n9HRi4m6JuGIV1eFlmvJB7ZEVxu93IrMyiMsGC0lrMJMWzRgx6WGquyfQgZVb31vhGgXnfmPNNXmxnOkRBrg==", "dev": true - }, - "send": { - "version": "0.18.0", - "resolved": "https://registry.npmjs.org/send/-/send-0.18.0.tgz", - "integrity": "sha512-qqWzuOjSFOuqPjFe4NOsMLafToQQwBSOEpS+FwEt3A2V3vKubTquT3vmLTQpFgMXp8AlFWFuP1qKaJZOtPpVXg==", - "dev": true, - "requires": { - "debug": "2.6.9", - "depd": "2.0.0", - "destroy": "1.2.0", - "encodeurl": "~1.0.2", - "escape-html": "~1.0.3", - "etag": "~1.8.1", - "fresh": "0.5.2", - "http-errors": "2.0.0", - "mime": "1.6.0", - "ms": "2.1.3", - "on-finished": "2.4.1", - "range-parser": "~1.2.1", - "statuses": "2.0.1" - } } } }, diff --git a/dashboard/package.json b/dashboard/package.json index 79897d63b19c8..66cdc9477fad6 100644 --- a/dashboard/package.json +++ b/dashboard/package.json @@ -63,7 +63,7 @@ "eslint-plugin-n": "^15.2.5", "eslint-plugin-promise": "^6.0.1", "eslint-plugin-react": "^7.31.6", - "express": "^4.20.0", + "express": "^4.21.1", "prettier": "^2.7.1", "prettier-plugin-organize-imports": "^3.1.1", "typescript": "5.4.2" From c5c2119901fa0ff17fd56a878293359f770b1f14 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Thu, 14 Nov 2024 16:12:21 +0800 Subject: [PATCH 09/16] fix(ci): delete unused python install before embedded UDF tests (#19386) Signed-off-by: Richard Chien --- ci/scripts/run-e2e-test.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index b3956998bd0e9..a8601fbb0ebef 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -136,7 +136,6 @@ sqllogictest -p 4566 -d dev './e2e_test/udf/external_udf.slt' pkill java echo "--- e2e, $mode, embedded udf" -python3 -m pip install --break-system-packages flask waitress sqllogictest -p 4566 -d dev './e2e_test/udf/wasm_udf.slt' sqllogictest -p 4566 -d dev './e2e_test/udf/rust_udf.slt' sqllogictest -p 4566 -d dev './e2e_test/udf/js_udf.slt' From 9aded7178aefb5436009b4cebd5e8f46b59429b2 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Thu, 14 Nov 2024 17:00:59 +0800 Subject: [PATCH 10/16] refactor(meta): merge drop and cancel streaming job command (#19369) --- src/meta/src/barrier/checkpoint/control.rs | 28 ++++++++------ src/meta/src/barrier/command.rs | 39 ++++++++++---------- src/meta/src/barrier/context/context_impl.rs | 19 ---------- src/meta/src/barrier/progress.rs | 2 +- src/meta/src/barrier/schedule.rs | 25 ++++--------- src/meta/src/stream/stream_manager.rs | 11 +++--- 6 files changed, 52 insertions(+), 72 deletions(-) diff --git a/src/meta/src/barrier/checkpoint/control.rs b/src/meta/src/barrier/checkpoint/control.rs index beb77b3217ad7..31150554cc686 100644 --- a/src/meta/src/barrier/checkpoint/control.rs +++ b/src/meta/src/barrier/checkpoint/control.rs @@ -758,20 +758,26 @@ impl DatabaseCheckpointControl { (None, vec![]) }; - if let Some(table_to_cancel) = command.as_ref().and_then(Command::table_to_cancel) - && self + for table_to_cancel in command + .as_ref() + .map(Command::tables_to_drop) + .into_iter() + .flatten() + { + if self .creating_streaming_job_controls .contains_key(&table_to_cancel) - { - warn!( - table_id = table_to_cancel.table_id, - "ignore cancel command on creating streaming job" - ); - for notifier in notifiers { - notifier - .notify_start_failed(anyhow!("cannot cancel creating streaming job, the job will continue creating until created or recovery").into()); + { + warn!( + table_id = table_to_cancel.table_id, + "ignore cancel command on creating streaming job" + ); + for notifier in notifiers { + notifier + .notify_start_failed(anyhow!("cannot cancel creating streaming job, the job will continue creating until created or recovery").into()); + } + return Ok(()); } - return Ok(()); } if let Some(Command::RescheduleFragment { .. }) = &command { diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 73ebc8d446295..d2dd3058544c4 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -232,6 +232,7 @@ pub enum Command { /// After the barrier is collected, it notifies the local stream manager of compute nodes to /// drop actors, and then delete the table fragments info from meta store. DropStreamingJobs { + table_fragments_ids: HashSet, actors: Vec, unregistered_state_table_ids: HashSet, unregistered_fragment_ids: HashSet, @@ -253,11 +254,6 @@ pub enum Command { MergeSnapshotBackfillStreamingJobs( HashMap, ), - /// `CancelStreamingJob` command generates a `Stop` barrier including the actors of the given - /// table fragment. - /// - /// The collecting and cleaning part works exactly the same as `DropStreamingJobs` command. - CancelStreamingJob(TableFragments), /// `Reschedule` command generates a `Update` barrier by the [`Reschedule`] of each fragment. /// Mainly used for scaling and migration. @@ -313,6 +309,18 @@ impl Command { Self::Resume(reason) } + pub fn cancel(table_fragments: &TableFragments) -> Self { + Self::DropStreamingJobs { + table_fragments_ids: HashSet::from_iter([table_fragments.table_id()]), + actors: table_fragments.actor_ids(), + unregistered_state_table_ids: table_fragments + .all_table_ids() + .map(TableId::new) + .collect(), + unregistered_fragment_ids: table_fragments.fragment_ids().collect(), + } + } + pub(crate) fn fragment_changes(&self) -> Option> { match self { Command::Flush => None, @@ -352,13 +360,6 @@ impl Command { Some(changes) } - Command::CancelStreamingJob(table_fragments) => Some( - table_fragments - .fragments - .values() - .map(|fragment| (fragment.fragment_id, CommandFragmentChanges::RemoveFragment)) - .collect(), - ), Command::RescheduleFragment { reschedules, .. } => Some( reschedules .iter() @@ -726,11 +727,6 @@ impl Command { })) } - Command::CancelStreamingJob(table_fragments) => { - let actors = table_fragments.actor_ids(); - Some(Mutation::Stop(StopMutation { actors })) - } - Command::ReplaceTable(ReplaceTablePlan { old_table_fragments, merge_updates, @@ -1013,10 +1009,15 @@ impl Command { } /// For `CancelStreamingJob`, returns the table id of the target table. - pub fn table_to_cancel(&self) -> Option { + pub fn tables_to_drop(&self) -> impl Iterator + '_ { match self { - Command::CancelStreamingJob(table_fragments) => Some(table_fragments.table_id()), + Command::DropStreamingJobs { + table_fragments_ids, + .. + } => Some(table_fragments_ids.iter().cloned()), _ => None, } + .into_iter() + .flatten() } } diff --git a/src/meta/src/barrier/context/context_impl.rs b/src/meta/src/barrier/context/context_impl.rs index 947c8a08ad3f1..fee2a31550cce 100644 --- a/src/meta/src/barrier/context/context_impl.rs +++ b/src/meta/src/barrier/context/context_impl.rs @@ -15,7 +15,6 @@ use std::sync::Arc; use futures::future::try_join_all; -use risingwave_common::catalog::TableId; use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::meta::PausedReason; @@ -163,24 +162,6 @@ impl CommandContext { .await?; } - Command::CancelStreamingJob(table_fragments) => { - tracing::debug!(id = ?table_fragments.table_id(), "cancelling stream job"); - - // NOTE(kwannoel): At this point, meta has already registered the table ids. - // We should unregister them. - // This is required for background ddl, for foreground ddl this is a no-op. - // Foreground ddl is handled entirely by stream manager, so it will unregister - // the table ids on failure. - // On the other hand background ddl could be handled by barrier manager. - // It won't clean the tables on failure, - // since the failure could be recoverable. - // As such it needs to be handled here. - barrier_manager_context - .hummock_manager - .unregister_table_ids(table_fragments.all_table_ids().map(TableId::new)) - .await?; - } - Command::CreateStreamingJob { info, job_type } => { let CreateStreamingJobCommandInfo { table_fragments, diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 36d1a9a0b242c..a40d526bc9ee2 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -447,7 +447,7 @@ impl CreateMviewProgressTracker { .flat_map(|resp| resp.create_mview_progress.iter()), version_stats, ); - if let Some(table_id) = command.and_then(Command::table_to_cancel) { + for table_id in command.map(Command::tables_to_drop).into_iter().flatten() { // the cancelled command is possibly stashed in `finished_commands` and waiting // for checkpoint, we should also clear it. self.cancel_command(table_id); diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index ebffb56efe5a4..2b3b78ede2f70 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashSet, VecDeque}; +use std::collections::VecDeque; use std::iter::once; use std::sync::Arc; use std::time::{Duration, Instant}; @@ -32,7 +32,6 @@ use super::notifier::Notifier; use super::{Command, Scheduled}; use crate::barrier::context::GlobalBarrierWorkerContext; use crate::hummock::HummockManagerRef; -use crate::model::ActorId; use crate::rpc::metrics::MetaMetrics; use crate::{MetaError, MetaResult}; @@ -106,9 +105,7 @@ impl ScheduledQueue { if let QueueStatus::Blocked(reason) = &self.status && !matches!( scheduled.command, - Command::DropStreamingJobs { .. } - | Command::CancelStreamingJob(_) - | Command::DropSubscription { .. } + Command::DropStreamingJobs { .. } | Command::DropSubscription { .. } ) { return Err(MetaError::unavailable(reason)); @@ -400,9 +397,7 @@ impl ScheduledBarriers { impl ScheduledBarriers { /// Pre buffered drop and cancel command, return true if any. pub(super) fn pre_apply_drop_cancel(&self) -> bool { - let (dropped_actors, cancelled) = self.pre_apply_drop_cancel_scheduled(); - - !dropped_actors.is_empty() || !cancelled.is_empty() + self.pre_apply_drop_cancel_scheduled() } /// Mark command scheduler as blocked and abort all queued scheduled command and notify with @@ -425,22 +420,18 @@ impl ScheduledBarriers { /// Try to pre apply drop and cancel scheduled command and return them if any. /// It should only be called in recovery. - pub(super) fn pre_apply_drop_cancel_scheduled(&self) -> (Vec, HashSet) { + pub(super) fn pre_apply_drop_cancel_scheduled(&self) -> bool { let mut queue = self.inner.queue.lock(); assert_matches!(queue.status, QueueStatus::Blocked(_)); - let (mut dropped_actors, mut cancel_table_ids) = (vec![], HashSet::new()); + let mut applied = false; while let Some(ScheduledQueueItem { notifiers, command, .. }) = queue.queue.pop_front() { match command { - Command::DropStreamingJobs { actors, .. } => { - dropped_actors.extend(actors); - } - Command::CancelStreamingJob(table_fragments) => { - let table_id = table_fragments.table_id(); - cancel_table_ids.insert(table_id); + Command::DropStreamingJobs { .. } => { + applied = true; } Command::DropSubscription { .. } => {} _ => { @@ -451,7 +442,7 @@ impl ScheduledBarriers { notify.notify_collected(); }); } - (dropped_actors, cancel_table_ids) + applied } } diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 509fffefd99b9..d15a73ecfa9c3 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -299,10 +299,7 @@ impl GlobalStreamManager { .await?; self.barrier_scheduler - .run_command( - database_id, - Command::CancelStreamingJob(table_fragments), - ) + .run_command(database_id, Command::cancel(&table_fragments)) .await?; } else { // streaming job is already completed. @@ -514,6 +511,10 @@ impl GlobalStreamManager { .run_command( database_id, Command::DropStreamingJobs { + table_fragments_ids: streaming_job_ids + .iter() + .map(|job_id| TableId::new(*job_id as _)) + .collect(), actors: removed_actors, unregistered_state_table_ids: state_table_ids .into_iter() @@ -576,7 +577,7 @@ impl GlobalStreamManager { if let Some(database_id) = database_id { self.barrier_scheduler - .run_command(DatabaseId::new(database_id as _), Command::CancelStreamingJob(fragment)) + .run_command(DatabaseId::new(database_id as _), Command::cancel(&fragment)) .await?; } }; From 1a97b4cccf266594477e487c063de76f2a085944 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 14 Nov 2024 17:35:07 +0800 Subject: [PATCH 11/16] feat(frontend): support iceberg predicate pushdown (#19228) --- ci/scripts/e2e-iceberg-sink-v2-test.sh | 1 + ci/workflows/main-cron.yml | 3 +- ci/workflows/pull-request.yml | 2 +- .../iceberg/start_spark_connect_server.sh | 2 +- .../test_case/iceberg_predicate_pushdown.slt | 143 ++++++++ .../test_case/iceberg_predicate_pushdown.toml | 11 + src/connector/src/source/iceberg/mod.rs | 42 ++- src/frontend/src/optimizer/mod.rs | 8 + .../src/optimizer/plan_node/batch_filter.rs | 6 + .../optimizer/plan_node/batch_iceberg_scan.rs | 46 ++- .../batch/batch_iceberg_predicate_pushdown.rs | 305 ++++++++++++++++++ src/frontend/src/optimizer/rule/batch/mod.rs | 1 + src/frontend/src/optimizer/rule/mod.rs | 2 + src/frontend/src/scheduler/plan_fragmenter.rs | 55 +++- 14 files changed, 599 insertions(+), 28 deletions(-) create mode 100644 e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt create mode 100644 e2e_test/iceberg/test_case/iceberg_predicate_pushdown.toml create mode 100644 src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs diff --git a/ci/scripts/e2e-iceberg-sink-v2-test.sh b/ci/scripts/e2e-iceberg-sink-v2-test.sh index bcb530ae9fdd9..27fc92a789d1b 100755 --- a/ci/scripts/e2e-iceberg-sink-v2-test.sh +++ b/ci/scripts/e2e-iceberg-sink-v2-test.sh @@ -49,6 +49,7 @@ poetry run python main.py -t ./test_case/iceberg_select_empty_table.toml poetry run python main.py -t ./test_case/iceberg_source_equality_delete.toml poetry run python main.py -t ./test_case/iceberg_source_position_delete.toml poetry run python main.py -t ./test_case/iceberg_source_all_delete.toml +poetry run python main.py -t ./test_case/iceberg_predicate_pushdown.toml echo "--- Kill cluster" diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 3f3cd705f09f9..e8a0fa32f1010 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -414,14 +414,13 @@ steps: depends_on: - "build" - "build-other" - plugins: - docker-compose#v5.1.0: run: rw-build-env config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 7 + timeout_in_minutes: 9 retry: *auto-retry - label: "end-to-end iceberg sink v2 test (release)" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 2991397dfb1c5..e10ffb2d0091f 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -270,7 +270,7 @@ steps: config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 15 + timeout_in_minutes: 17 retry: *auto-retry - label: "end-to-end iceberg cdc test" diff --git a/e2e_test/iceberg/start_spark_connect_server.sh b/e2e_test/iceberg/start_spark_connect_server.sh index 8f0c2640a1b59..7996899f7a4d1 100755 --- a/e2e_test/iceberg/start_spark_connect_server.sh +++ b/e2e_test/iceberg/start_spark_connect_server.sh @@ -11,7 +11,7 @@ PACKAGES="$PACKAGES,org.apache.spark:spark-connect_2.12:$SPARK_VERSION" SPARK_FILE="spark-${SPARK_VERSION}-bin-hadoop3.tgz" if [ ! -d "spark-${SPARK_VERSION}-bin-hadoop3" ];then - wget https://dlcdn.apache.org/spark/spark-${SPARK_VERSION}/$SPARK_FILE + wget --no-verbose https://dlcdn.apache.org/spark/spark-${SPARK_VERSION}/$SPARK_FILE tar -xzf $SPARK_FILE --no-same-owner fi diff --git a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt new file mode 100644 index 0000000000000..2075d129a8a1e --- /dev/null +++ b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt @@ -0,0 +1,143 @@ +statement ok +set sink_decouple = false; + +statement ok +set streaming_parallelism=4; + +statement ok +drop table if exists s1 cascade; + +statement ok +CREATE TABLE s1 (i1 int, i2 varchar, i3 varchar); + +statement ok +insert into s1 select x, 'some str', 'another str' from generate_series(1, 500) t(x); + +statement ok +insert into s1 select x, null as y, null as z from generate_series(501, 1000) t(x); + +statement ok +flush; + +statement ok +CREATE MATERIALIZED VIEW mv1 AS SELECT * FROM s1; + +statement ok +CREATE SINK sink1 AS select * from mv1 WITH ( + connector = 'iceberg', + type = 'append-only', + force_append_only = 'true', + database.name = 'demo_db', + table.name = 't1', + catalog.name = 'demo', + catalog.type = 'storage', + warehouse.path = 's3a://icebergdata/demo', + s3.endpoint = 'http://127.0.0.1:9301', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + commit_checkpoint_interval = 1, + create_table_if_not_exists = 'true' +); + +statement ok +drop source if exists iceberg_t1_source; + +statement ok +CREATE SOURCE iceberg_t1_source +WITH ( + connector = 'iceberg', + s3.endpoint = 'http://127.0.0.1:9301', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + s3.path.style.access = 'true', + catalog.type = 'storage', + warehouse.path = 's3a://icebergdata/demo', + database.name = 'demo_db', + table.name = 't1', +); + +statement ok +flush; + +query I +select * from iceberg_t1_source order by i1 limit 1; +---- +1 some str another str + +query I +select count(*) from iceberg_t1_source; +---- +1000 + +query I +select * from iceberg_t1_source where i1 > 990 order by i1; +---- +991 NULL NULL +992 NULL NULL +993 NULL NULL +994 NULL NULL +995 NULL NULL +996 NULL NULL +997 NULL NULL +998 NULL NULL +999 NULL NULL +1000 NULL NULL + +query I +explain select * from iceberg_t1_source where i1 > 500 and i1 < 600 and i1 >= 550 and i1 <= 590 and i1 != 570 and i1 = 580; +---- + BatchExchange { order: [], dist: Single } + └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1, i2, i3], predicate: (((((i1 = 580) AND (i1 > 500)) AND (i1 < 600)) AND (i1 >= 550)) AND (i1 <= 590)) AND (i1 != 570) } + +query I +select i1 from iceberg_t1_source where i1 > 500 and i1 < 600 and i1 >= 550 and i1 <= 590 and i1 != 570 and i1 = 580; +---- +580 + +query I +explain select * from iceberg_t1_source where i1 in (1, 2, 3, 4, 5); +---- + BatchExchange { order: [], dist: Single } + └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1, i2, i3], predicate: i1 IN (5, 4, 1, 3, 2) } + +query I +select i1 from iceberg_t1_source where i1 in (1, 2, 3, 4, 5) order by i1; +---- +1 +2 +3 +4 +5 + +query I +select count(*), i2, i3 from iceberg_t1_source where i2 = 'some str' and i3 = 'another str' group by i2, i3; +---- +500 some str another str + +query I +explain select i1 from iceberg_t1_source where i1 > 500 and i2 = i3; +---- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [i1] } + └─BatchFilter { predicate: (i2 = i3) } + └─BatchIcebergScan { source: iceberg_t1_source, columns: [i1, i2, i3], predicate: i1 > 500 } + +query I +select i1 from iceberg_t1_source where i1 > 500 and i2 = i3; +---- + +# Empty splits should not panic +query I +select i1 from iceberg_t1_source where i1 > 1001; +---- + +statement ok +DROP SINK sink1; + +statement ok +DROP SOURCE iceberg_t1_source; + +statement ok +DROP TABLE s1 cascade; diff --git a/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.toml b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.toml new file mode 100644 index 0000000000000..c08dcbb827db1 --- /dev/null +++ b/e2e_test/iceberg/test_case/iceberg_predicate_pushdown.toml @@ -0,0 +1,11 @@ +init_sqls = [ + 'CREATE SCHEMA IF NOT EXISTS demo_db', + 'DROP TABLE IF EXISTS demo_db.t1', +] + +slt = 'test_case/iceberg_predicate_pushdown.slt' + +drop_sqls = [ + 'DROP TABLE IF EXISTS demo_db.t1', + 'DROP SCHEMA IF EXISTS demo_db', +] diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index 60a26e43e1d31..aeb642c80a014 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -19,6 +19,7 @@ use std::collections::HashMap; use anyhow::anyhow; use async_trait::async_trait; use futures_async_stream::for_await; +use iceberg::expr::Predicate as IcebergPredicate; use iceberg::scan::FileScanTask; use iceberg::spec::TableMetadata; use iceberg::table::Table; @@ -137,6 +138,19 @@ pub struct IcebergSplit { pub position_delete_files: Vec, } +impl IcebergSplit { + pub fn empty(table_meta: TableMetadataJsonStr) -> Self { + Self { + split_id: 0, + snapshot_id: 0, + table_meta, + files: vec![], + equality_delete_files: vec![], + position_delete_files: vec![], + } + } +} + impl SplitMetaData for IcebergSplit { fn id(&self) -> SplitId { self.split_id.to_string().into() @@ -189,6 +203,7 @@ impl IcebergSplitEnumerator { schema: Schema, time_traval_info: Option, batch_parallelism: usize, + predicate: IcebergPredicate, ) -> ConnectorResult> { if batch_parallelism == 0 { bail!("Batch parallelism is 0. Cannot split the iceberg files."); @@ -199,14 +214,9 @@ impl IcebergSplitEnumerator { let current_snapshot = table.metadata().current_snapshot(); if current_snapshot.is_none() { // If there is no snapshot, we will return a mock `IcebergSplit` with empty files. - return Ok(vec![IcebergSplit { - split_id: 0, - snapshot_id: 0, - table_meta: TableMetadataJsonStr::serialize(table.metadata()), - files: vec![], - equality_delete_files: vec![], - position_delete_files: vec![], - }]); + return Ok(vec![IcebergSplit::empty(TableMetadataJsonStr::serialize( + table.metadata(), + ))]); } let snapshot_id = match time_traval_info { @@ -246,11 +256,15 @@ impl IcebergSplitEnumerator { let require_names = Self::get_require_field_names(&table, snapshot_id, &schema).await?; + let table_schema = table.metadata().current_schema(); + tracing::debug!("iceberg_table_schema: {:?}", table_schema); + let mut position_delete_files = vec![]; let mut data_files = vec![]; let mut equality_delete_files = vec![]; let scan = table .scan() + .with_filter(predicate) .snapshot_id(snapshot_id) .select(require_names) .build() @@ -302,10 +316,18 @@ impl IcebergSplitEnumerator { .files .push(data_files[split_num * split_size + i].clone()); } - Ok(splits + let splits = splits .into_iter() .filter(|split| !split.files.is_empty()) - .collect_vec()) + .collect_vec(); + + if splits.is_empty() { + return Ok(vec![IcebergSplit::empty(TableMetadataJsonStr::serialize( + table.metadata(), + ))]); + } + + Ok(splits) } /// The required field names are the intersection of the output shema and the equality delete columns. diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index e08f6b2c4dd4a..30d51bb93326c 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -402,6 +402,14 @@ impl PlanRoot { ApplyOrder::BottomUp, )); + // For iceberg scan, we do iceberg predicate pushdown + // BatchFilter -> BatchIcebergScan + let plan = plan.optimize_by_rules(&OptimizationStage::new( + "Iceberg Predicate Pushdown", + vec![BatchIcebergPredicatePushDownRule::create()], + ApplyOrder::BottomUp, + )); + assert_eq!(plan.convention(), Convention::Batch); Ok(plan) } diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index ff89eacd485c0..6404fd852e6d6 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -45,6 +45,12 @@ impl BatchFilter { pub fn predicate(&self) -> &Condition { &self.core.predicate } + + pub fn clone_with_predicate(&self, predicate: Condition) -> Self { + let mut core = self.core.clone(); + core.predicate = predicate; + Self::new(core) + } } impl_distill_by_unit!(BatchFilter, core, "BatchFilter"); diff --git a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs index 4333fcaa3e90a..815b711faa296 100644 --- a/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::hash::{Hash, Hasher}; use std::rc::Rc; +use iceberg::expr::Predicate as IcebergPredicate; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::IcebergScanNode; @@ -29,10 +31,36 @@ use crate::error::Result; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::property::{Distribution, Order}; -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone)] pub struct BatchIcebergScan { pub base: PlanBase, pub core: generic::Source, + pub predicate: IcebergPredicate, +} + +impl PartialEq for BatchIcebergScan { + fn eq(&self, other: &Self) -> bool { + if self.predicate == IcebergPredicate::AlwaysTrue + && other.predicate == IcebergPredicate::AlwaysTrue + { + self.base == other.base && self.core == other.core + } else { + panic!("BatchIcebergScan::eq: comparing non-AlwaysTrue predicates is not supported") + } + } +} + +impl Eq for BatchIcebergScan {} + +impl Hash for BatchIcebergScan { + fn hash(&self, state: &mut H) { + if self.predicate != IcebergPredicate::AlwaysTrue { + panic!("BatchIcebergScan::hash: hashing non-AlwaysTrue predicates is not supported") + } else { + self.base.hash(state); + self.core.hash(state); + } + } } impl BatchIcebergScan { @@ -44,7 +72,11 @@ impl BatchIcebergScan { Order::any(), ); - Self { base, core } + Self { + base, + core, + predicate: IcebergPredicate::AlwaysTrue, + } } pub fn column_names(&self) -> Vec<&str> { @@ -62,6 +94,15 @@ impl BatchIcebergScan { Self { base, core: self.core.clone(), + predicate: self.predicate.clone(), + } + } + + pub fn clone_with_predicate(&self, predicate: IcebergPredicate) -> Self { + Self { + base: self.base.clone(), + core: self.core.clone(), + predicate, } } @@ -78,6 +119,7 @@ impl Distill for BatchIcebergScan { let fields = vec![ ("source", src), ("columns", column_names_pretty(self.schema())), + ("predicate", Pretty::from(self.predicate.to_string())), ]; childless_record("BatchIcebergScan", fields) } diff --git a/src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs b/src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs new file mode 100644 index 0000000000000..8df8777d59384 --- /dev/null +++ b/src/frontend/src/optimizer/rule/batch/batch_iceberg_predicate_pushdown.rs @@ -0,0 +1,305 @@ +// 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. +// +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +use chrono::Datelike; +use iceberg::expr::{Predicate as IcebergPredicate, Reference}; +use iceberg::spec::Datum as IcebergDatum; +use risingwave_common::catalog::Field; +use risingwave_common::types::{Decimal, ScalarImpl}; + +use crate::expr::{ExprImpl, ExprType, Literal}; +use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::{BatchFilter, BatchIcebergScan, PlanTreeNodeUnary}; +use crate::optimizer::rule::{BoxedRule, Rule}; +use crate::optimizer::PlanRef; +use crate::utils::Condition; + +/// NOTE(kwannoel): We do predicate pushdown to the iceberg-sdk here. +/// zone-map is used to evaluate predicates on iceberg tables. +/// Without zone-map, iceberg-sdk will still apply the predicate on its own. +/// See: . +pub struct BatchIcebergPredicatePushDownRule {} + +impl Rule for BatchIcebergPredicatePushDownRule { + fn apply(&self, plan: PlanRef) -> Option { + let filter: &BatchFilter = plan.as_batch_filter()?; + let input = filter.input(); + let scan: &BatchIcebergScan = input.as_batch_iceberg_scan()?; + // NOTE(kwannoel): We only fill iceberg predicate here. + assert_eq!(scan.predicate, IcebergPredicate::AlwaysTrue); + + let predicate = filter.predicate().clone(); + let (iceberg_predicate, rw_predicate) = + rw_predicate_to_iceberg_predicate(predicate, scan.schema().fields()); + let scan = scan.clone_with_predicate(iceberg_predicate); + if rw_predicate.always_true() { + Some(scan.into()) + } else { + let filter = filter + .clone_with_input(scan.into()) + .clone_with_predicate(rw_predicate); + Some(filter.into()) + } + } +} + +fn rw_literal_to_iceberg_datum(literal: &Literal) -> Option { + let Some(scalar) = literal.get_data() else { + return None; + }; + match scalar { + ScalarImpl::Bool(b) => Some(IcebergDatum::bool(*b)), + ScalarImpl::Int32(i) => Some(IcebergDatum::int(*i)), + ScalarImpl::Int64(i) => Some(IcebergDatum::long(*i)), + ScalarImpl::Float32(f) => Some(IcebergDatum::float(*f)), + ScalarImpl::Float64(f) => Some(IcebergDatum::double(*f)), + ScalarImpl::Decimal(d) => { + let Decimal::Normalized(d) = d else { + return None; + }; + let Ok(d) = IcebergDatum::decimal(*d) else { + return None; + }; + Some(d) + } + ScalarImpl::Date(d) => { + let Ok(datum) = IcebergDatum::date_from_ymd(d.0.year(), d.0.month(), d.0.day()) else { + return None; + }; + Some(datum) + } + ScalarImpl::Timestamp(t) => Some(IcebergDatum::timestamp_nanos( + t.0.and_utc().timestamp_nanos_opt()?, + )), + ScalarImpl::Timestamptz(t) => Some(IcebergDatum::timestamptz_micros(t.timestamp_micros())), + ScalarImpl::Utf8(s) => Some(IcebergDatum::string(s)), + ScalarImpl::Bytea(b) => Some(IcebergDatum::binary(b.clone())), + _ => None, + } +} + +fn rw_expr_to_iceberg_predicate(expr: &ExprImpl, fields: &[Field]) -> Option { + match expr { + ExprImpl::Literal(l) => match l.get_data() { + Some(ScalarImpl::Bool(b)) => { + if *b { + Some(IcebergPredicate::AlwaysTrue) + } else { + Some(IcebergPredicate::AlwaysFalse) + } + } + _ => None, + }, + ExprImpl::FunctionCall(f) => { + let args = f.inputs(); + match f.func_type() { + ExprType::Not => { + let arg = rw_expr_to_iceberg_predicate(&args[0], fields)?; + Some(IcebergPredicate::negate(arg)) + } + ExprType::And => { + let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; + let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; + Some(IcebergPredicate::and(arg0, arg1)) + } + ExprType::Or => { + let arg0 = rw_expr_to_iceberg_predicate(&args[0], fields)?; + let arg1 = rw_expr_to_iceberg_predicate(&args[1], fields)?; + Some(IcebergPredicate::or(arg0, arg1)) + } + ExprType::Equal => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] + | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.equal_to(datum)) + } + _ => None, + }, + ExprType::NotEqual => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] + | [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.not_equal_to(datum)) + } + _ => None, + }, + ExprType::GreaterThan => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than_or_equal_to(datum)) + } + _ => None, + }, + ExprType::GreaterThanOrEqual => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than_or_equal_to(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than(datum)) + } + _ => None, + }, + ExprType::LessThan => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than_or_equal_to(datum)) + } + _ => None, + }, + ExprType::LessThanOrEqual => match [&args[0], &args[1]] { + [ExprImpl::InputRef(lhs), ExprImpl::Literal(rhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.less_than_or_equal_to(datum)) + } + [ExprImpl::Literal(rhs), ExprImpl::InputRef(lhs)] => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let datum = rw_literal_to_iceberg_datum(rhs)?; + Some(reference.greater_than(datum)) + } + _ => None, + }, + ExprType::IsNull => match &args[0] { + ExprImpl::InputRef(lhs) => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + Some(reference.is_null()) + } + _ => None, + }, + ExprType::IsNotNull => match &args[0] { + ExprImpl::InputRef(lhs) => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + Some(reference.is_not_null()) + } + _ => None, + }, + ExprType::In => match &args[0] { + ExprImpl::InputRef(lhs) => { + let column_name = &fields[lhs.index].name; + let reference = Reference::new(column_name); + let mut datums = Vec::with_capacity(args.len() - 1); + for arg in &args[1..] { + if let ExprImpl::Literal(l) = arg { + if let Some(datum) = rw_literal_to_iceberg_datum(l) { + datums.push(datum); + } else { + return None; + } + } else { + return None; + } + } + Some(reference.is_in(datums)) + } + _ => None, + }, + _ => None, + } + } + _ => None, + } +} +fn rw_predicate_to_iceberg_predicate( + predicate: Condition, + fields: &[Field], +) -> (IcebergPredicate, Condition) { + if predicate.always_true() { + return (IcebergPredicate::AlwaysTrue, predicate); + } + + let mut conjunctions = predicate.conjunctions; + let mut ignored_conjunctions: Vec = Vec::with_capacity(conjunctions.len()); + + let mut iceberg_condition_root = None; + while let Some(conjunction) = conjunctions.pop() { + match rw_expr_to_iceberg_predicate(&conjunction, fields) { + iceberg_predicate @ Some(_) => { + iceberg_condition_root = iceberg_predicate; + break; + } + None => { + ignored_conjunctions.push(conjunction); + continue; + } + } + } + + let mut iceberg_condition_root = match iceberg_condition_root { + Some(p) => p, + None => { + return ( + IcebergPredicate::AlwaysTrue, + Condition { + conjunctions: ignored_conjunctions, + }, + ) + } + }; + + for rw_condition in conjunctions { + match rw_expr_to_iceberg_predicate(&rw_condition, fields) { + Some(iceberg_predicate) => { + iceberg_condition_root = iceberg_condition_root.and(iceberg_predicate) + } + None => ignored_conjunctions.push(rw_condition), + } + } + ( + iceberg_condition_root, + Condition { + conjunctions: ignored_conjunctions, + }, + ) +} + +impl BatchIcebergPredicatePushDownRule { + pub fn create() -> BoxedRule { + Box::new(BatchIcebergPredicatePushDownRule {}) + } +} diff --git a/src/frontend/src/optimizer/rule/batch/mod.rs b/src/frontend/src/optimizer/rule/batch/mod.rs index 6061c985b6696..c4d31faf3cfba 100644 --- a/src/frontend/src/optimizer/rule/batch/mod.rs +++ b/src/frontend/src/optimizer/rule/batch/mod.rs @@ -12,5 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod batch_iceberg_predicate_pushdown; pub(crate) mod batch_project_merge_rule; pub mod batch_push_limit_to_scan_rule; diff --git a/src/frontend/src/optimizer/rule/mod.rs b/src/frontend/src/optimizer/rule/mod.rs index 7468f1c96524c..e9bd08e6c6797 100644 --- a/src/frontend/src/optimizer/rule/mod.rs +++ b/src/frontend/src/optimizer/rule/mod.rs @@ -165,6 +165,7 @@ mod table_function_to_mysql_query_rule; mod table_function_to_postgres_query_rule; mod values_extract_project_rule; +pub use batch::batch_iceberg_predicate_pushdown::*; pub use batch::batch_push_limit_to_scan_rule::*; pub use pull_up_correlated_predicate_agg_rule::*; pub use source_to_iceberg_scan_rule::*; @@ -248,6 +249,7 @@ macro_rules! for_all_rules { , { AggCallMergeRule } , { ValuesExtractProjectRule } , { BatchPushLimitToScanRule } + , { BatchIcebergPredicatePushDownRule } , { PullUpCorrelatedPredicateAggRule } , { SourceToKafkaScanRule } , { SourceToIcebergScanRule } diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index 90984750bc460..9cec27601a246 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -22,6 +22,7 @@ use anyhow::anyhow; use async_recursion::async_recursion; use enum_as_inner::EnumAsInner; use futures::TryStreamExt; +use iceberg::expr::Predicate as IcebergPredicate; use itertools::Itertools; use pgwire::pg_server::SessionId; use risingwave_batch::error::BatchError; @@ -268,11 +269,25 @@ impl Query { } } +#[derive(Debug, Clone)] +pub enum SourceFetchParameters { + IcebergPredicate(IcebergPredicate), + KafkaTimebound { + lower: Option, + upper: Option, + }, + Empty, +} + #[derive(Debug, Clone)] pub struct SourceFetchInfo { pub schema: Schema, + /// These are user-configured connector properties. + /// e.g. host, username, etc... pub connector: ConnectorProperties, - pub timebound: (Option, Option), + /// These parameters are internally derived by the plan node. + /// e.g. predicate pushdown for iceberg, timebound for kafka. + pub fetch_parameters: SourceFetchParameters, pub as_of: Option, } @@ -295,13 +310,16 @@ impl SourceScanInfo { unreachable!("Never call complete when SourceScanInfo is already complete") } }; - match fetch_info.connector { - ConnectorProperties::Kafka(prop) => { + match (fetch_info.connector, fetch_info.fetch_parameters) { + ( + ConnectorProperties::Kafka(prop), + SourceFetchParameters::KafkaTimebound { lower, upper }, + ) => { let mut kafka_enumerator = KafkaSplitEnumerator::new(*prop, SourceEnumeratorContext::dummy().into()) .await?; let split_info = kafka_enumerator - .list_splits_batch(fetch_info.timebound.0, fetch_info.timebound.1) + .list_splits_batch(lower, upper) .await? .into_iter() .map(SplitImpl::Kafka) @@ -309,7 +327,7 @@ impl SourceScanInfo { Ok(SourceScanInfo::Complete(split_info)) } - ConnectorProperties::OpendalS3(prop) => { + (ConnectorProperties::OpendalS3(prop), SourceFetchParameters::Empty) => { let lister: OpendalEnumerator = OpendalEnumerator::new_s3_source(prop.s3_properties, prop.assume_role)?; let stream = build_opendal_fs_list_for_batch(lister); @@ -322,7 +340,7 @@ impl SourceScanInfo { Ok(SourceScanInfo::Complete(res)) } - ConnectorProperties::Gcs(prop) => { + (ConnectorProperties::Gcs(prop), SourceFetchParameters::Empty) => { let lister: OpendalEnumerator = OpendalEnumerator::new_gcs_source(*prop)?; let stream = build_opendal_fs_list_for_batch(lister); @@ -331,7 +349,7 @@ impl SourceScanInfo { Ok(SourceScanInfo::Complete(res)) } - ConnectorProperties::Azblob(prop) => { + (ConnectorProperties::Azblob(prop), SourceFetchParameters::Empty) => { let lister: OpendalEnumerator = OpendalEnumerator::new_azblob_source(*prop)?; let stream = build_opendal_fs_list_for_batch(lister); @@ -340,7 +358,10 @@ impl SourceScanInfo { Ok(SourceScanInfo::Complete(res)) } - ConnectorProperties::Iceberg(prop) => { + ( + ConnectorProperties::Iceberg(prop), + SourceFetchParameters::IcebergPredicate(predicate), + ) => { let iceberg_enumerator = IcebergSplitEnumerator::new(*prop, SourceEnumeratorContext::dummy().into()) .await?; @@ -369,7 +390,12 @@ impl SourceScanInfo { }; let split_info = iceberg_enumerator - .list_splits_batch(fetch_info.schema, time_travel_info, batch_parallelism) + .list_splits_batch( + fetch_info.schema, + time_travel_info, + batch_parallelism, + predicate, + ) .await? .into_iter() .map(SplitImpl::Iceberg) @@ -1068,7 +1094,10 @@ impl BatchPlanFragmenter { return Ok(Some(SourceScanInfo::new(SourceFetchInfo { schema: batch_kafka_scan.base.schema().clone(), connector: property, - timebound: timestamp_bound, + fetch_parameters: SourceFetchParameters::KafkaTimebound { + lower: timestamp_bound.0, + upper: timestamp_bound.1, + }, as_of: None, }))); } @@ -1082,7 +1111,9 @@ impl BatchPlanFragmenter { return Ok(Some(SourceScanInfo::new(SourceFetchInfo { schema: batch_iceberg_scan.base.schema().clone(), connector: property, - timebound: (None, None), + fetch_parameters: SourceFetchParameters::IcebergPredicate( + batch_iceberg_scan.predicate.clone(), + ), as_of, }))); } @@ -1097,7 +1128,7 @@ impl BatchPlanFragmenter { return Ok(Some(SourceScanInfo::new(SourceFetchInfo { schema: source_node.base.schema().clone(), connector: property, - timebound: (None, None), + fetch_parameters: SourceFetchParameters::Empty, as_of, }))); } From fa99969ebbfbd46476455d7dc33a553aa18ab68a Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 14 Nov 2024 17:41:11 +0800 Subject: [PATCH 12/16] fix(pg-cdc): write transactional WAL message as heartbeat action (#19385) --- .../src/main/resources/postgres.properties | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/postgres.properties b/java/connector-node/risingwave-connector-service/src/main/resources/postgres.properties index 89701280b246a..3922ad2fdf479 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/postgres.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/postgres.properties @@ -19,9 +19,9 @@ publication.autocreate.mode=disabled publication.name=${publication.name:-rw_publication} # default heartbeat interval 5 mins heartbeat.interval.ms=${debezium.heartbeat.interval.ms:-300000} -# emit a WAL message to the replication stream +# emit a transactional WAL message to the replication stream # see https://github.com/risingwavelabs/risingwave/issues/16697 for more details -heartbeat.action.query=SELECT pg_logical_emit_message(false, 'heartbeat', now()::varchar) +heartbeat.action.query=SELECT pg_logical_emit_message(true, 'heartbeat', now()::varchar) # In sharing cdc source mode, we will subscribe to multiple tables in the given database, # so here we set ${table.name} to a default value `RW_CDC_Sharing` just for display. name=${hostname}:${port}:${database.name}.${schema.name}.${table.name:-RW_CDC_Sharing} From c1435dd7bc1f5281bee14a2845e925c9f5e76347 Mon Sep 17 00:00:00 2001 From: August Date: Thu, 14 Nov 2024 17:46:32 +0800 Subject: [PATCH 13/16] fix: fix system acl column type to support `\l` command (#19379) --- .../batch/catalog/slash_l_database.slt.part | 17 +++++++++++++ .../src/catalog/system_catalog/mod.rs | 25 ++++++++----------- .../system_catalog/pg_catalog/pg_database.rs | 2 +- .../system_catalog/pg_catalog/pg_namespace.rs | 2 +- .../pg_catalog/pg_tablespace.rs | 2 +- .../rw_catalog/rw_connections.rs | 4 +-- .../system_catalog/rw_catalog/rw_databases.rs | 2 +- .../system_catalog/rw_catalog/rw_functions.rs | 2 +- .../system_catalog/rw_catalog/rw_indexes.rs | 4 +-- .../rw_catalog/rw_internal_tables.rs | 2 +- .../rw_catalog/rw_materialized_views.rs | 2 +- .../system_catalog/rw_catalog/rw_relations.rs | 2 +- .../system_catalog/rw_catalog/rw_schemas.rs | 2 +- .../system_catalog/rw_catalog/rw_secrets.rs | 4 +-- .../system_catalog/rw_catalog/rw_sinks.rs | 2 +- .../system_catalog/rw_catalog/rw_sources.rs | 2 +- .../rw_catalog/rw_subscriptions.rs | 2 +- .../rw_catalog/rw_system_tables.rs | 2 +- .../system_catalog/rw_catalog/rw_tables.rs | 2 +- .../system_catalog/rw_catalog/rw_views.rs | 2 +- 20 files changed, 48 insertions(+), 36 deletions(-) create mode 100644 e2e_test/batch/catalog/slash_l_database.slt.part diff --git a/e2e_test/batch/catalog/slash_l_database.slt.part b/e2e_test/batch/catalog/slash_l_database.slt.part new file mode 100644 index 0000000000000..1b1d463029cce --- /dev/null +++ b/e2e_test/batch/catalog/slash_l_database.slt.part @@ -0,0 +1,17 @@ +# wrapped test of `\l` command for better consistency. +query T +SELECT count(*) > 0 +FROM +(SELECT + d.datname AS "Name", + pg_catalog.pg_get_userbyid (d.datdba) AS "Owner", + pg_catalog.pg_encoding_to_char (d.encoding) AS "Encoding", + d.datcollate AS "Collate", + d.datctype AS "Ctype", + pg_catalog.array_to_string (d.datacl, E'\n') AS "Access privileges" +FROM + pg_catalog.pg_database AS d +ORDER BY + 1); +---- +t diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 432266d1871a9..b374567c4372e 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -237,9 +237,8 @@ fn get_acl_items( for_dml_table: bool, users: &Vec, username_map: &HashMap, -) -> String { - let mut res = String::from("{"); - let mut empty_flag = true; +) -> Vec { + let mut res = vec![]; let super_privilege = available_prost_privilege(*object, for_dml_table); for user in users { let privileges = if user.is_super { @@ -263,25 +262,21 @@ fn get_acl_items( }) }); for (granted_by, actions) in grantor_map { - if empty_flag { - empty_flag = false; - } else { - res.push(','); - } - res.push_str(&user.name); - res.push('='); + let mut aclitem = String::new(); + aclitem.push_str(&user.name); + aclitem.push('='); for (action, option) in actions { - res.push_str(&AclMode::from(action).to_string()); + aclitem.push_str(&AclMode::from(action).to_string()); if option { - res.push('*'); + aclitem.push('*'); } } - res.push('/'); + aclitem.push('/'); // should be able to query grantor's name - res.push_str(username_map.get(&granted_by).unwrap()); + aclitem.push_str(username_map.get(&granted_by).unwrap()); + res.push(aclitem); } } - res.push('}'); res } diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs index b0510d0244345..183200af8a491 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs @@ -58,5 +58,5 @@ struct PgDatabase { datallowconn: bool, datconnlimit: i32, dattablespace: i32, - datacl: String, + datacl: Vec, } diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs index 69506da1ecd67..e01023f75d795 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs @@ -28,5 +28,5 @@ struct PgNamespace { oid: i32, nspname: String, nspowner: i32, - nspacl: String, + nspacl: Vec, } diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs index 81ef923bfa24e..f6d35011a27f1 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs @@ -24,6 +24,6 @@ struct PgTablespace { oid: i32, spcname: String, spcowner: i32, - spcacl: String, + spcacl: Vec, spcoptions: String, } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs index 2af0b29b16f76..fcc7e8efc3389 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs @@ -27,7 +27,7 @@ struct RwConnection { owner: i32, type_: String, provider: String, - acl: String, + acl: Vec, } #[system_catalog(table, "rw_catalog.rw_connections")] @@ -44,7 +44,7 @@ fn read_rw_connections(reader: &SysCatalogReaderImpl) -> Result, } #[system_catalog(table, "rw_catalog.rw_databases")] diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs index 9f002dcab6f16..ce34bfcef42b2 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs @@ -31,7 +31,7 @@ struct RwFunction { return_type_id: i32, language: String, link: Option, - acl: String, + acl: Vec, always_retry_on_network_error: bool, } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs index 558e628a3fbfc..b1d42a1ba0c65 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs @@ -29,7 +29,7 @@ struct RwIndex { schema_id: i32, owner: i32, definition: String, - acl: String, + acl: Vec, initialized_at: Option, created_at: Option, initialized_at_cluster_version: Option, @@ -76,7 +76,7 @@ fn read_rw_indexes(reader: &SysCatalogReaderImpl) -> Result> { schema_id: schema.id() as i32, owner: index.index_table.owner as i32, definition: index.index_table.create_sql(), - acl: "".into(), + acl: vec![], initialized_at: index.initialized_at_epoch.map(|e| e.as_timestamptz()), created_at: index.created_at_epoch.map(|e| e.as_timestamptz()), initialized_at_cluster_version: index.initialized_at_cluster_version.clone(), diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs index 9ea91bfa50731..989226d104a5e 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs @@ -27,7 +27,7 @@ struct RwInternalTable { schema_id: i32, owner: i32, definition: String, - acl: String, + acl: Vec, initialized_at: Option, created_at: Option, initialized_at_cluster_version: Option, diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs index a0e8d98b24b69..a1ce41737ab80 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs @@ -29,7 +29,7 @@ struct RwMaterializedView { owner: i32, definition: String, append_only: bool, - acl: String, + acl: Vec, initialized_at: Option, created_at: Option, initialized_at_cluster_version: Option, diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relations.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relations.rs index abe39dbc329a6..a500020e62896 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relations.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relations.rs @@ -37,5 +37,5 @@ struct RwRelation { schema_id: i32, owner: i32, definition: String, - acl: String, + acl: Vec, } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs index 8d8786e0b1098..adaf6563d4aef 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs @@ -26,7 +26,7 @@ struct RwSchema { id: i32, name: String, owner: i32, - acl: String, + acl: Vec, } #[system_catalog(table, "rw_catalog.rw_schemas")] diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_secrets.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_secrets.rs index 09c9a98798e06..33a43c3de51f1 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_secrets.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_secrets.rs @@ -24,7 +24,7 @@ struct RwSecret { id: i32, name: String, owner: i32, - acl: String, + acl: Vec, } #[system_catalog(table, "rw_catalog.rw_secrets")] @@ -38,7 +38,7 @@ fn read_rw_view_info(reader: &SysCatalogReaderImpl) -> Result> { id: secret.id.secret_id() as i32, name: secret.name.clone(), owner: secret.owner as i32, - acl: "".into(), + acl: vec![], }) }) .collect()) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs index a7bee63805fce..e382a5b7dfaf3 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs @@ -31,7 +31,7 @@ struct RwSink { sink_type: String, connection_id: Option, definition: String, - acl: String, + acl: Vec, initialized_at: Option, created_at: Option, initialized_at_cluster_version: Option, diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs index 40df3dfc3a849..bdcfe355057a4 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs @@ -35,7 +35,7 @@ struct RwSource { associated_table_id: Option, connection_id: Option, definition: String, - acl: String, + acl: Vec, initialized_at: Option, created_at: Option, initialized_at_cluster_version: Option, diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_subscriptions.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_subscriptions.rs index 95d22630475ad..c5c1e108e3ef5 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_subscriptions.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_subscriptions.rs @@ -27,7 +27,7 @@ struct RwSubscription { schema_id: i32, owner: i32, definition: String, - acl: String, + acl: Vec, initialized_at: Option, created_at: Option, initialized_at_cluster_version: Option, diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs index 0fda14d726876..17d3001e3263c 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs @@ -28,7 +28,7 @@ struct SystemTable { schema_id: i32, owner: i32, definition: Option, - acl: String, + acl: Vec, } #[system_catalog(table, "rw_catalog.rw_system_tables")] diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs index 78416c97b71af..d991315d26b72 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs @@ -28,7 +28,7 @@ struct RwTable { owner: i32, definition: String, append_only: bool, - acl: String, + acl: Vec, initialized_at: Option, created_at: Option, initialized_at_cluster_version: Option, diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs index 7c156d783a1c3..2141f808362fe 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs @@ -27,7 +27,7 @@ struct RwView { schema_id: i32, owner: i32, definition: String, - acl: String, + acl: Vec, } #[system_catalog(table, "rw_catalog.rw_views")] From daed1f243da61ad8132f6f14a702dce6936b825d Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 14 Nov 2024 18:20:02 +0800 Subject: [PATCH 14/16] feat(iceberg): make wrehouse.path optional for iceberg rest catalog (#19380) --- .../src/connector_common/iceberg/mod.rs | 122 +++++++++++------- src/connector/src/sink/iceberg/mod.rs | 26 ++-- src/connector/with_options_sink.yaml | 2 +- src/connector/with_options_source.yaml | 2 +- 4 files changed, 97 insertions(+), 55 deletions(-) diff --git a/src/connector/src/connector_common/iceberg/mod.rs b/src/connector/src/connector_common/iceberg/mod.rs index 3d1b016577c6d..d10a9eefb68aa 100644 --- a/src/connector/src/connector_common/iceberg/mod.rs +++ b/src/connector/src/connector_common/iceberg/mod.rs @@ -49,7 +49,7 @@ pub struct IcebergCommon { pub secret_key: String, /// Path of iceberg warehouse, only applicable in storage catalog. #[serde(rename = "warehouse.path")] - pub warehouse_path: String, + pub warehouse_path: Option, /// Catalog name, can be omitted for storage catalog, but /// must be set for other catalogs. #[serde(rename = "catalog.name")] @@ -142,23 +142,31 @@ impl IcebergCommon { self.secret_key.clone().to_string(), ); - let (bucket, _) = { - let url = Url::parse(&self.warehouse_path) - .with_context(|| format!("Invalid warehouse path: {}", self.warehouse_path))?; - let bucket = url - .host_str() - .with_context(|| { - format!( - "Invalid s3 path: {}, bucket is missing", - self.warehouse_path - ) - })? - .to_string(); - let root = url.path().trim_start_matches('/').to_string(); - (bucket, root) - }; + match &self.warehouse_path { + Some(warehouse_path) => { + let (bucket, _) = { + let url = Url::parse(warehouse_path).with_context(|| { + format!("Invalid warehouse path: {}", warehouse_path) + })?; + let bucket = url + .host_str() + .with_context(|| { + format!("Invalid s3 path: {}, bucket is missing", warehouse_path) + })? + .to_string(); + let root = url.path().trim_start_matches('/').to_string(); + (bucket, root) + }; + + iceberg_configs.insert("iceberg.table.io.bucket".to_string(), bucket); + } + None => { + if catalog_type != "rest" { + bail!("`warehouse.path` must be set in {} catalog", &catalog_type); + } + } + } - iceberg_configs.insert("iceberg.table.io.bucket".to_string(), bucket); // #TODO // Support load config file iceberg_configs.insert( @@ -176,7 +184,9 @@ impl IcebergCommon { java_catalog_configs.insert("uri".to_string(), uri.to_string()); } - java_catalog_configs.insert("warehouse".to_string(), self.warehouse_path.clone()); + if let Some(warehouse_path) = &self.warehouse_path { + java_catalog_configs.insert("warehouse".to_string(), warehouse_path.clone()); + } java_catalog_configs.extend(java_catalog_props.clone()); // Currently we only support s3, so let's set it to s3 @@ -241,6 +251,7 @@ impl IcebergCommon { /// icelake mod v1 { + use anyhow::anyhow; use icelake::catalog::{load_catalog, CatalogRef}; use icelake::{Table, TableIdentifier}; @@ -268,7 +279,9 @@ mod v1 { "storage" => { iceberg_configs.insert( format!("iceberg.catalog.{}.warehouse", self.catalog_name()), - self.warehouse_path.clone(), + self.warehouse_path.clone().ok_or_else(|| { + anyhow!("`warehouse.path` must be set in storage catalog") + })?, ); iceberg_configs.insert(CATALOG_TYPE.to_string(), "storage".into()); } @@ -318,28 +331,36 @@ mod v1 { ); } - let (bucket, root) = { - let url = Url::parse(&self.warehouse_path) - .with_context(|| format!("Invalid warehouse path: {}", self.warehouse_path))?; - let bucket = url - .host_str() - .with_context(|| { - format!( - "Invalid s3 path: {}, bucket is missing", - self.warehouse_path - ) - })? - .to_string(); - let root = url.path().trim_start_matches('/').to_string(); - (bucket, root) - }; + match &self.warehouse_path { + Some(warehouse_path) => { + let (bucket, root) = { + let url = Url::parse(warehouse_path).with_context(|| { + format!("Invalid warehouse path: {}", warehouse_path) + })?; + let bucket = url + .host_str() + .with_context(|| { + format!("Invalid s3 path: {}, bucket is missing", warehouse_path) + })? + .to_string(); + let root = url.path().trim_start_matches('/').to_string(); + (bucket, root) + }; - iceberg_configs.insert("iceberg.table.io.bucket".to_string(), bucket); + iceberg_configs.insert("iceberg.table.io.bucket".to_string(), bucket); - // Only storage catalog should set this. - if catalog_type == "storage" { - iceberg_configs.insert("iceberg.table.io.root".to_string(), root); + // Only storage catalog should set this. + if catalog_type == "storage" { + iceberg_configs.insert("iceberg.table.io.root".to_string(), root); + } + } + None => { + if catalog_type == "storage" { + bail!("`warehouse.path` must be set in storage catalog"); + } + } } + // #TODO // Support load config file iceberg_configs.insert( @@ -416,6 +437,7 @@ mod v1 { /// iceberg-rust mod v2 { + use anyhow::anyhow; use iceberg::spec::TableMetadata; use iceberg::table::Table as TableV2; use iceberg::{Catalog as CatalogV2, TableIdent}; @@ -442,7 +464,9 @@ mod v2 { match self.catalog_type() { "storage" => { let config = storage_catalog::StorageCatalogConfig::builder() - .warehouse(self.warehouse_path.clone()) + .warehouse(self.warehouse_path.clone().ok_or_else(|| { + anyhow!("`warehouse.path` must be set in storage catalog") + })?) .access_key(self.access_key.clone()) .secret_key(self.secret_key.clone()) .region(self.region.clone()) @@ -468,12 +492,18 @@ mod v2 { S3_SECRET_ACCESS_KEY.to_string(), self.secret_key.clone().to_string(), ); - let config = iceberg_catalog_rest::RestCatalogConfig::builder() + let config_builder = iceberg_catalog_rest::RestCatalogConfig::builder() .uri(self.catalog_uri.clone().with_context(|| { "`catalog.uri` must be set in rest catalog".to_string() })?) - .props(iceberg_configs) - .build(); + .props(iceberg_configs); + + let config = match &self.warehouse_path { + Some(warehouse_path) => { + config_builder.warehouse(warehouse_path.clone()).build() + } + None => config_builder.build(), + }; let catalog = iceberg_catalog_rest::RestCatalog::new(config); Ok(Arc::new(catalog)) } @@ -509,7 +539,9 @@ mod v2 { self.secret_key.clone().to_string(), ); let config_builder = iceberg_catalog_glue::GlueCatalogConfig::builder() - .warehouse(self.warehouse_path.clone()) + .warehouse(self.warehouse_path.clone().ok_or_else(|| { + anyhow!("`warehouse.path` must be set in glue catalog") + })?) .props(iceberg_configs); let config = if let Some(uri) = self.catalog_uri.as_deref() { config_builder.uri(uri.to_string()).build() @@ -575,7 +607,9 @@ mod v2 { match self.catalog_type() { "storage" => { let config = storage_catalog::StorageCatalogConfig::builder() - .warehouse(self.warehouse_path.clone()) + .warehouse(self.warehouse_path.clone().ok_or_else(|| { + anyhow!("`warehouse.path` must be set in storage catalog") + })?) .access_key(self.access_key.clone()) .secret_key(self.secret_key.clone()) .region(self.region.clone()) diff --git a/src/connector/src/sink/iceberg/mod.rs b/src/connector/src/sink/iceberg/mod.rs index e2f00afd5b525..0c878ae1ba6d6 100644 --- a/src/connector/src/sink/iceberg/mod.rs +++ b/src/connector/src/sink/iceberg/mod.rs @@ -281,18 +281,26 @@ impl IcebergSink { let location = { let mut names = namespace.clone().inner(); names.push(self.config.common.table_name.to_string()); - if self.config.common.warehouse_path.ends_with('/') { - format!("{}{}", self.config.common.warehouse_path, names.join("/")) - } else { - format!("{}/{}", self.config.common.warehouse_path, names.join("/")) + match &self.config.common.warehouse_path { + Some(warehouse_path) => { + if warehouse_path.ends_with('/') { + Some(format!("{}{}", warehouse_path, names.join("/"))) + } else { + Some(format!("{}/{}", warehouse_path, names.join("/"))) + } + } + None => None, } }; - let table_creation = TableCreation::builder() + let table_creation_builder = TableCreation::builder() .name(self.config.common.table_name.clone()) - .schema(iceberg_schema) - .location(location) - .build(); + .schema(iceberg_schema); + + let table_creation = match location { + Some(location) => table_creation_builder.location(location).build(), + None => table_creation_builder.build(), + }; catalog .create_table(&namespace, table_creation) @@ -998,7 +1006,7 @@ mod test { let expected_iceberg_config = IcebergConfig { common: IcebergCommon { - warehouse_path: "s3://iceberg".to_string(), + warehouse_path: Some("s3://iceberg".to_string()), catalog_uri: Some("jdbc://postgresql://postgres:5432/iceberg".to_string()), region: Some("us-east-1".to_string()), endpoint: Some("http://127.0.0.1:9301".to_string()), diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index 2c286695b312c..31579dfd70325 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -394,7 +394,7 @@ IcebergConfig: - name: warehouse.path field_type: String comments: Path of iceberg warehouse, only applicable in storage catalog. - required: true + required: false - name: catalog.name field_type: String comments: |- diff --git a/src/connector/with_options_source.yaml b/src/connector/with_options_source.yaml index 41c27a1af7eb6..75972546b2994 100644 --- a/src/connector/with_options_source.yaml +++ b/src/connector/with_options_source.yaml @@ -99,7 +99,7 @@ IcebergProperties: - name: warehouse.path field_type: String comments: Path of iceberg warehouse, only applicable in storage catalog. - required: true + required: false - name: catalog.name field_type: String comments: |- From c92069481d2aba5a77ffb673b48c7e27b5a698a6 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Thu, 14 Nov 2024 18:43:01 +0800 Subject: [PATCH 15/16] refactor(barrier): add control request to explicitly create partial graph (#19383) --- proto/stream_service.proto | 12 +++- src/meta/src/barrier/checkpoint/control.rs | 18 +++--- src/meta/src/barrier/context/context_impl.rs | 6 +- src/meta/src/barrier/context/mod.rs | 4 +- src/meta/src/barrier/rpc.rs | 61 ++++++++++++++----- src/meta/src/barrier/worker.rs | 7 ++- src/rpc_client/src/stream_client.rs | 9 +-- src/stream/src/task/barrier_manager.rs | 52 ++++++++++++---- .../src/task/barrier_manager/managed_state.rs | 51 ++++++++++------ src/stream/src/task/stream_manager.rs | 8 ++- 10 files changed, 160 insertions(+), 68 deletions(-) diff --git a/proto/stream_service.proto b/proto/stream_service.proto index 8b006626cd741..62cc8746aeca1 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -63,10 +63,19 @@ message WaitEpochCommitResponse { } message StreamingControlStreamRequest { - message InitRequest { + message InitialPartialGraph { + uint64 partial_graph_id = 1; repeated stream_plan.SubscriptionUpstreamInfo subscriptions = 2; } + message InitRequest { + repeated InitialPartialGraph graphs = 1; + } + + message CreatePartialGraphRequest { + uint64 partial_graph_id = 1; + } + message RemovePartialGraphRequest { repeated uint64 partial_graph_ids = 1; } @@ -75,6 +84,7 @@ message StreamingControlStreamRequest { InitRequest init = 1; InjectBarrierRequest inject_barrier = 2; RemovePartialGraphRequest remove_partial_graph = 3; + CreatePartialGraphRequest create_partial_graph = 4; } } diff --git a/src/meta/src/barrier/checkpoint/control.rs b/src/meta/src/barrier/checkpoint/control.rs index 31150554cc686..1489738c2f9e2 100644 --- a/src/meta/src/barrier/checkpoint/control.rs +++ b/src/meta/src/barrier/checkpoint/control.rs @@ -24,7 +24,6 @@ use risingwave_meta_model::WorkerId; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::meta::PausedReason; -use risingwave_pb::stream_plan::PbSubscriptionUpstreamInfo; use risingwave_pb::stream_service::BarrierCompleteResponse; use tracing::{debug, warn}; @@ -40,7 +39,7 @@ use crate::barrier::schedule::{NewBarrier, PeriodicBarriers}; use crate::barrier::utils::collect_creating_job_commit_epoch_info; use crate::barrier::{ BarrierKind, Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, - SnapshotBackfillInfo, TracedEpoch, + InflightSubscriptionInfo, SnapshotBackfillInfo, TracedEpoch, }; use crate::manager::ActiveStreamingWorkerNodes; use crate::rpc::metrics::GLOBAL_META_METRICS; @@ -147,6 +146,7 @@ impl CheckpointControl { } else { new_database.state.in_flight_prev_epoch().clone() }; + control_stream_manager.add_partial_graph(database_id, None)?; (entry.insert(new_database), max_prev_epoch) } Command::Flush @@ -276,10 +276,12 @@ impl CheckpointControl { .for_each(|database| database.create_mview_tracker.abort_all()); } - pub(crate) fn subscriptions(&self) -> impl Iterator + '_ { - self.databases - .values() - .flat_map(|database| &database.state.inflight_subscription_info) + pub(crate) fn subscriptions( + &self, + ) -> impl Iterator + '_ { + self.databases.iter().map(|(database_id, database)| { + (*database_id, &database.state.inflight_subscription_info) + }) } } @@ -828,8 +830,10 @@ impl DatabaseCheckpointControl { .expect("checked Some") .to_mutation(None) .expect("should have some mutation in `CreateStreamingJob` command"); + let job_id = info.table_fragments.table_id(); + control_stream_manager.add_partial_graph(self.database_id, Some(job_id))?; self.creating_streaming_job_controls.insert( - info.table_fragments.table_id(), + job_id, CreatingStreamingJobControl::new( info.clone(), snapshot_backfill_info.clone(), diff --git a/src/meta/src/barrier/context/context_impl.rs b/src/meta/src/barrier/context/context_impl.rs index fee2a31550cce..2ccdee2782861 100644 --- a/src/meta/src/barrier/context/context_impl.rs +++ b/src/meta/src/barrier/context/context_impl.rs @@ -18,7 +18,7 @@ use futures::future::try_join_all; use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::meta::PausedReason; -use risingwave_pb::stream_plan::SubscriptionUpstreamInfo; +use risingwave_pb::stream_service::streaming_control_stream_request::PbInitRequest; use risingwave_pb::stream_service::WaitEpochCommitRequest; use risingwave_rpc_client::StreamingControlHandle; @@ -70,9 +70,9 @@ impl GlobalBarrierWorkerContext for GlobalBarrierWorkerContextImpl { async fn new_control_stream( &self, node: &WorkerNode, - subscriptions: impl Iterator, + init_request: &PbInitRequest, ) -> MetaResult { - self.new_control_stream_impl(node, subscriptions).await + self.new_control_stream_impl(node, init_request).await } async fn reload_runtime_info(&self) -> MetaResult { diff --git a/src/meta/src/barrier/context/mod.rs b/src/meta/src/barrier/context/mod.rs index e69b9644de8dd..7306c16171621 100644 --- a/src/meta/src/barrier/context/mod.rs +++ b/src/meta/src/barrier/context/mod.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use arc_swap::ArcSwap; use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::HummockVersionStats; -use risingwave_pb::stream_plan::SubscriptionUpstreamInfo; +use risingwave_pb::stream_service::streaming_control_stream_request::PbInitRequest; use risingwave_rpc_client::StreamingControlHandle; use crate::barrier::command::CommandContext; @@ -60,7 +60,7 @@ pub(super) trait GlobalBarrierWorkerContext: Send + Sync + 'static { async fn new_control_stream( &self, node: &WorkerNode, - subscriptions: impl Iterator, + init_request: &PbInitRequest, ) -> MetaResult; async fn reload_runtime_info(&self) -> MetaResult; diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index ea84625b19331..dfb9f1cc13d37 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -30,7 +30,9 @@ use risingwave_pb::common::{ActorInfo, WorkerNode}; use risingwave_pb::meta::PausedReason; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::{Barrier, BarrierMutation, StreamActor, SubscriptionUpstreamInfo}; -use risingwave_pb::stream_service::streaming_control_stream_request::RemovePartialGraphRequest; +use risingwave_pb::stream_service::streaming_control_stream_request::{ + CreatePartialGraphRequest, PbInitRequest, PbInitialPartialGraph, RemovePartialGraphRequest, +}; use risingwave_pb::stream_service::{ streaming_control_stream_request, streaming_control_stream_response, BarrierCompleteResponse, InjectBarrierRequest, StreamingControlStreamRequest, @@ -94,7 +96,7 @@ impl ControlStreamManager { pub(super) async fn add_worker( &mut self, node: WorkerNode, - subscriptions: impl Iterator, + initial_subscriptions: impl Iterator, context: &impl GlobalBarrierWorkerContext, ) { let node_id = node.id as WorkerId; @@ -106,13 +108,10 @@ impl ControlStreamManager { let mut backoff = ExponentialBackoff::from_millis(100) .max_delay(Duration::from_secs(3)) .factor(5); - let subscriptions = subscriptions.collect_vec(); + let init_request = Self::collect_init_request(initial_subscriptions); const MAX_RETRY: usize = 5; for i in 1..=MAX_RETRY { - match context - .new_control_stream(&node, subscriptions.iter().cloned()) - .await - { + match context.new_control_stream(&node, &init_request).await { Ok(handle) => { assert!(self .nodes @@ -141,16 +140,14 @@ impl ControlStreamManager { pub(super) async fn reset( &mut self, - subscriptions: impl Iterator, + initial_subscriptions: impl Iterator, nodes: &HashMap, context: &impl GlobalBarrierWorkerContext, ) -> MetaResult<()> { - let subscriptions = subscriptions.cloned().collect_vec(); - let subscriptions = &subscriptions; + let init_request = Self::collect_init_request(initial_subscriptions); + let init_request = &init_request; let nodes = try_join_all(nodes.iter().map(|(worker_id, node)| async move { - let handle = context - .new_control_stream(node, subscriptions.iter().flatten()) - .await?; + let handle = context.new_control_stream(node, init_request).await?; Result::<_, MetaError>::Ok(( *worker_id, ControlStreamNode { @@ -270,6 +267,19 @@ impl ControlStreamManager { tracing::debug!(?errors, "collected stream errors"); errors } + + fn collect_init_request( + initial_subscriptions: impl Iterator, + ) -> PbInitRequest { + PbInitRequest { + graphs: initial_subscriptions + .map(|(database_id, info)| PbInitialPartialGraph { + partial_graph_id: to_partial_graph_id(database_id, None), + subscriptions: info.into_iter().collect_vec(), + }) + .collect(), + } + } } impl ControlStreamManager { @@ -436,6 +446,27 @@ impl ControlStreamManager { Ok(node_need_collect) } + pub(super) fn add_partial_graph( + &mut self, + database_id: DatabaseId, + creating_job_id: Option, + ) -> MetaResult<()> { + let partial_graph_id = to_partial_graph_id(database_id, creating_job_id); + self.nodes.iter().try_for_each(|(_, node)| { + node.handle + .request_sender + .send(StreamingControlStreamRequest { + request: Some( + streaming_control_stream_request::Request::CreatePartialGraph( + CreatePartialGraphRequest { partial_graph_id }, + ), + ), + }) + .map_err(|_| anyhow!("failed to add partial graph")) + })?; + Ok(()) + } + pub(super) fn remove_partial_graph( &mut self, database_id: DatabaseId, @@ -472,14 +503,14 @@ impl GlobalBarrierWorkerContextImpl { pub(super) async fn new_control_stream_impl( &self, node: &WorkerNode, - subscriptions: impl Iterator, + init_request: &PbInitRequest, ) -> MetaResult { let handle = self .env .stream_client_pool() .get(node) .await? - .start_streaming_control(subscriptions) + .start_streaming_control(init_request.clone()) .await?; Ok(handle) } diff --git a/src/meta/src/barrier/worker.rs b/src/meta/src/barrier/worker.rs index ee0cdd97fb616..f6999e5ce9235 100644 --- a/src/meta/src/barrier/worker.rs +++ b/src/meta/src/barrier/worker.rs @@ -14,7 +14,7 @@ use std::collections::HashMap; use std::mem::replace; -use std::sync::Arc; +use std::sync::{Arc, LazyLock}; use std::time::Duration; use arc_swap::ArcSwap; @@ -46,7 +46,7 @@ use crate::barrier::rpc::{merge_node_rpc_errors, ControlStreamManager}; use crate::barrier::schedule::PeriodicBarriers; use crate::barrier::{ schedule, BarrierKind, BarrierManagerRequest, BarrierManagerStatus, - BarrierWorkerRuntimeInfoSnapshot, RecoveryReason, TracedEpoch, + BarrierWorkerRuntimeInfoSnapshot, InflightSubscriptionInfo, RecoveryReason, TracedEpoch, }; use crate::error::MetaErrorInner; use crate::hummock::HummockManagerRef; @@ -558,9 +558,10 @@ impl GlobalBarrierWorker { let mut control_stream_manager = ControlStreamManager::new(self.env.clone()); let reset_start_time = Instant::now(); + let empty_subscriptions = LazyLock::new(InflightSubscriptionInfo::default); control_stream_manager .reset( - subscription_infos.values(), + database_fragment_infos.keys().map(|database_id| (*database_id, subscription_infos.get(database_id).unwrap_or_else(|| &*empty_subscriptions))), active_streaming_nodes.current(), &*self.context, ) diff --git a/src/rpc_client/src/stream_client.rs b/src/rpc_client/src/stream_client.rs index 6484adb1c921f..9b83ab82d7fef 100644 --- a/src/rpc_client/src/stream_client.rs +++ b/src/rpc_client/src/stream_client.rs @@ -21,9 +21,8 @@ use futures::TryStreamExt; use risingwave_common::config::MAX_CONNECTION_WINDOW_SIZE; use risingwave_common::monitor::{EndpointExt, TcpConfig}; use risingwave_common::util::addr::HostAddr; -use risingwave_pb::stream_plan::SubscriptionUpstreamInfo; use risingwave_pb::stream_service::stream_service_client::StreamServiceClient; -use risingwave_pb::stream_service::streaming_control_stream_request::InitRequest; +use risingwave_pb::stream_service::streaming_control_stream_request::PbInitRequest; use risingwave_pb::stream_service::streaming_control_stream_response::InitResponse; use risingwave_pb::stream_service::*; use tokio_stream::wrappers::UnboundedReceiverStream; @@ -86,13 +85,11 @@ pub type StreamingControlHandle = impl StreamClient { pub async fn start_streaming_control( &self, - subscriptions: impl Iterator, + init_request: PbInitRequest, ) -> Result { let first_request = StreamingControlStreamRequest { request: Some(streaming_control_stream_request::Request::Init( - InitRequest { - subscriptions: subscriptions.collect(), - }, + init_request, )), }; let mut client = self.0.to_owned(); diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 0bfdcdd59117c..155320281421b 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -52,7 +52,9 @@ use risingwave_common::util::runtime::BackgroundShutdownRuntime; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; use risingwave_hummock_sdk::{LocalSstableInfo, SyncResult}; use risingwave_pb::stream_plan::barrier::BarrierKind; -use risingwave_pb::stream_service::streaming_control_stream_request::{InitRequest, Request}; +use risingwave_pb::stream_service::streaming_control_stream_request::{ + InitRequest, InitialPartialGraph, Request, +}; use risingwave_pb::stream_service::streaming_control_stream_response::{ InitResponse, ShutdownResponse, }; @@ -64,7 +66,9 @@ use risingwave_pb::stream_service::{ use crate::executor::exchange::permit::Receiver; use crate::executor::monitor::StreamingMetrics; use crate::executor::{Barrier, BarrierInner, StreamExecutorError}; -use crate::task::barrier_manager::managed_state::ManagedBarrierStateDebugInfo; +use crate::task::barrier_manager::managed_state::{ + ManagedBarrierStateDebugInfo, PartialGraphManagedBarrierState, +}; use crate::task::barrier_manager::progress::BackfillState; /// If enabled, all actors will be grouped in the same tracing span within one epoch. @@ -273,7 +277,10 @@ pub(super) struct LocalBarrierWorker { } impl LocalBarrierWorker { - pub(super) fn new(actor_manager: Arc) -> Self { + pub(super) fn new( + actor_manager: Arc, + initial_partial_graphs: Vec, + ) -> Self { let (event_tx, event_rx) = unbounded_channel(); let (failure_tx, failure_rx) = unbounded_channel(); let shared_context = Arc::new(SharedContext::new( @@ -284,7 +291,11 @@ impl LocalBarrierWorker { }, )); Self { - state: ManagedBarrierState::new(actor_manager.clone(), shared_context.clone()), + state: ManagedBarrierState::new( + actor_manager.clone(), + shared_context.clone(), + initial_partial_graphs, + ), control_stream_handle: ControlStreamHandle::empty(), actor_manager, current_shared_context: shared_context, @@ -327,8 +338,7 @@ impl LocalBarrierWorker { match actor_op { LocalActorOperation::NewControlStream { handle, init_request } => { self.control_stream_handle.reset_stream_with_err(Status::internal("control stream has been reset to a new one")); - self.reset().await; - self.state.add_subscriptions(init_request.subscriptions); + self.reset(init_request.graphs).await; self.control_stream_handle = handle; self.control_stream_handle.send_response(StreamingControlStreamResponse { response: Some(streaming_control_stream_response::Response::Init(InitResponse {})) @@ -379,6 +389,10 @@ impl LocalBarrierWorker { ); Ok(()) } + Request::CreatePartialGraph(req) => { + self.add_partial_graph(PartialGraphId::new(req.partial_graph_id)); + Ok(()) + } Request::Init(_) => { unreachable!() } @@ -557,9 +571,20 @@ impl LocalBarrierWorker { } } + pub(super) fn add_partial_graph(&mut self, partial_graph_id: PartialGraphId) { + assert!(self + .state + .graph_states + .insert( + partial_graph_id, + PartialGraphManagedBarrierState::new(&self.actor_manager) + ) + .is_none()); + } + /// Reset all internal states. - pub(super) fn reset_state(&mut self) { - *self = Self::new(self.actor_manager.clone()); + pub(super) fn reset_state(&mut self, initial_partial_graphs: Vec) { + *self = Self::new(self.actor_manager.clone(), initial_partial_graphs); } /// When a [`crate::executor::StreamConsumer`] (typically [`crate::executor::DispatchExecutor`]) get a barrier, it should report @@ -659,7 +684,7 @@ impl LocalBarrierWorker { await_tree_reg, runtime: runtime.into(), }); - let worker = LocalBarrierWorker::new(actor_manager); + let worker = LocalBarrierWorker::new(actor_manager, vec![]); tokio::spawn(worker.run(actor_op_rx)) } } @@ -842,7 +867,9 @@ pub(crate) mod barrier_test_utils { use assert_matches::assert_matches; use futures::StreamExt; - use risingwave_pb::stream_service::streaming_control_stream_request::InitRequest; + use risingwave_pb::stream_service::streaming_control_stream_request::{ + InitRequest, PbInitialPartialGraph, + }; use risingwave_pb::stream_service::{ streaming_control_stream_request, streaming_control_stream_response, InjectBarrierRequest, StreamingControlStreamRequest, StreamingControlStreamResponse, @@ -876,7 +903,10 @@ pub(crate) mod barrier_test_utils { UnboundedReceiverStream::new(request_rx).boxed(), ), init_request: InitRequest { - subscriptions: vec![], + graphs: vec![PbInitialPartialGraph { + partial_graph_id: u64::MAX, + subscriptions: vec![], + }], }, }); diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 462f78233f25b..cd6bb924f4784 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -142,6 +142,7 @@ mod await_epoch_completed_future { use await_epoch_completed_future::*; use risingwave_pb::stream_plan::SubscriptionUpstreamInfo; +use risingwave_pb::stream_service::streaming_control_stream_request::InitialPartialGraph; use risingwave_pb::stream_service::InjectBarrierRequest; fn sync_epoch( @@ -372,6 +373,8 @@ pub(super) struct PartialGraphManagedBarrierState { prev_barrier_table_ids: Option<(EpochPair, HashSet)>, + mv_depended_subscriptions: HashMap>, + /// Record the progress updates of creating mviews for each epoch of concurrent checkpoints. /// /// This is updated by [`super::CreateMviewProgressReporter::update`] and will be reported to meta @@ -390,7 +393,15 @@ pub(super) struct PartialGraphManagedBarrierState { } impl PartialGraphManagedBarrierState { - fn new( + pub(super) fn new(actor_manager: &StreamActorManager) -> Self { + Self::new_inner( + actor_manager.env.state_store(), + actor_manager.streaming_metrics.clone(), + actor_manager.await_tree_reg.clone(), + ) + } + + fn new_inner( state_store: StateStoreImpl, streaming_metrics: Arc, barrier_await_tree_reg: Option, @@ -398,6 +409,7 @@ impl PartialGraphManagedBarrierState { Self { epoch_barrier_state_map: Default::default(), prev_barrier_table_ids: None, + mv_depended_subscriptions: Default::default(), create_mview_progress: Default::default(), await_epoch_completed_futures: Default::default(), state_store, @@ -408,7 +420,7 @@ impl PartialGraphManagedBarrierState { #[cfg(test)] pub(crate) fn for_test() -> Self { - Self::new( + Self::new_inner( StateStoreImpl::for_test(), Arc::new(StreamingMetrics::unused()), None, @@ -425,8 +437,6 @@ pub(crate) struct ManagedBarrierState { pub(super) graph_states: HashMap, - mv_depended_subscriptions: HashMap>, - actor_manager: Arc, current_shared_context: Arc, @@ -437,11 +447,18 @@ impl ManagedBarrierState { pub(super) fn new( actor_manager: Arc, current_shared_context: Arc, + initial_partial_graphs: Vec, ) -> Self { Self { actor_states: Default::default(), - graph_states: Default::default(), - mv_depended_subscriptions: Default::default(), + graph_states: initial_partial_graphs + .into_iter() + .map(|graph| { + let mut state = PartialGraphManagedBarrierState::new(&actor_manager); + state.add_subscriptions(graph.subscriptions); + (PartialGraphId::new(graph.partial_graph_id), state) + }) + .collect(), actor_manager, current_shared_context, } @@ -506,7 +523,9 @@ impl ManagedBarrierState { .expect("should exist") .register_barrier_sender(tx) } +} +impl PartialGraphManagedBarrierState { pub(super) fn add_subscriptions(&mut self, subscriptions: Vec) { for subscription_to_add in subscriptions { if !self @@ -557,14 +576,14 @@ impl ManagedBarrierState { } } } +} +impl ManagedBarrierState { pub(super) fn transform_to_issued( &mut self, barrier: &Barrier, request: InjectBarrierRequest, ) -> StreamResult<()> { - self.add_subscriptions(request.subscriptions_to_add); - self.remove_subscriptions(request.subscriptions_to_remove); let partial_graph_id = PartialGraphId::new(request.partial_graph_id); let actor_to_stop = barrier.all_stop_actors(); let is_stop_actor = |actor_id| { @@ -574,14 +593,11 @@ impl ManagedBarrierState { }; let graph_state = self .graph_states - .entry(partial_graph_id) - .or_insert_with(|| { - PartialGraphManagedBarrierState::new( - self.actor_manager.env.state_store(), - self.actor_manager.streaming_metrics.clone(), - self.actor_manager.await_tree_reg.clone(), - ) - }); + .get_mut(&partial_graph_id) + .expect("should exist"); + + graph_state.add_subscriptions(request.subscriptions_to_add); + graph_state.remove_subscriptions(request.subscriptions_to_remove); graph_state.transform_to_issued( barrier, @@ -590,7 +606,8 @@ impl ManagedBarrierState { ); let mut new_actors = HashSet::new(); - let subscriptions = LazyCell::new(|| Arc::new(self.mv_depended_subscriptions.clone())); + let subscriptions = + LazyCell::new(|| Arc::new(graph_state.mv_depended_subscriptions.clone())); for actor in request.actors_to_build { let actor_id = actor.actor_id; assert!(!is_stop_actor(actor_id)); diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 361c5d9582d64..648afb81ebc8d 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -33,7 +33,9 @@ use risingwave_pb::plan_common::StorageTableDesc; use risingwave_pb::stream_plan; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{StreamActor, StreamNode, StreamScanNode, StreamScanType}; -use risingwave_pb::stream_service::streaming_control_stream_request::InitRequest; +use risingwave_pb::stream_service::streaming_control_stream_request::{ + InitRequest, InitialPartialGraph, +}; use risingwave_pb::stream_service::{ StreamingControlStreamRequest, StreamingControlStreamResponse, }; @@ -248,7 +250,7 @@ impl LocalStreamManager { impl LocalBarrierWorker { /// Force stop all actors on this worker, and then drop their resources. - pub(super) async fn reset(&mut self) { + pub(super) async fn reset(&mut self, initial_partial_graphs: Vec) { self.state.abort_actors().await; if let Some(m) = self.actor_manager.await_tree_reg.as_ref() { m.clear(); @@ -260,8 +262,8 @@ impl LocalBarrierWorker { .verbose_instrument_await("store_clear_shared_buffer") .await } - self.reset_state(); self.actor_manager.env.dml_manager_ref().clear(); + self.reset_state(initial_partial_graphs); } } From df07ffbce4a77f7f0a6335acfae99c0b7e7cccf0 Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 14 Nov 2024 20:33:46 +0800 Subject: [PATCH 16/16] feat(dashboard): show cluster version (#19388) Signed-off-by: xxchan --- dashboard/lib/api/cluster.ts | 5 +++++ dashboard/pages/cluster.tsx | 6 ++++++ src/meta/src/dashboard/mod.rs | 5 +++++ src/meta/src/manager/diagnose.rs | 5 +++-- 4 files changed, 19 insertions(+), 2 deletions(-) diff --git a/dashboard/lib/api/cluster.ts b/dashboard/lib/api/cluster.ts index 0efc083309389..a1977c7c51e4c 100644 --- a/dashboard/lib/api/cluster.ts +++ b/dashboard/lib/api/cluster.ts @@ -35,3 +35,8 @@ export async function getClusterInfoComputeNode() { ) return res } + +export async function getClusterVersion() { + const res = await api.get("/version") + return res +} diff --git a/dashboard/pages/cluster.tsx b/dashboard/pages/cluster.tsx index eabc07741bcb1..7f356905ef247 100644 --- a/dashboard/pages/cluster.tsx +++ b/dashboard/pages/cluster.tsx @@ -36,6 +36,7 @@ import { getClusterInfoComputeNode, getClusterInfoFrontend, getClusterMetrics, + getClusterVersion, } from "../lib/api/cluster" import { WorkerNode } from "../proto/gen/common" @@ -142,6 +143,7 @@ export default function Cluster() { const [frontendList, setFrontendList] = useState([]) const [computeNodeList, setComputeNodeList] = useState([]) const [metrics, setMetrics] = useState() + const [version, setVersion] = useState() const toast = useErrorToast() useEffect(() => { @@ -149,6 +151,7 @@ export default function Cluster() { try { setFrontendList(await getClusterInfoFrontend()) setComputeNodeList(await getClusterInfoComputeNode()) + setVersion(await getClusterVersion()) } catch (e: any) { toast(e) } @@ -182,6 +185,9 @@ export default function Cluster() { const retVal = ( Cluster Overview + + Version: {version} + {frontendList.map((frontend) => ( ) -> Result> { + Ok(Json(risingwave_common::current_cluster_version())) + } } impl DashboardService { @@ -505,6 +509,7 @@ impl DashboardService { .allow_methods(vec![Method::GET]); let api_router = Router::new() + .route("/version", get(get_version)) .route("/clusters/:ty", get(list_clusters)) .route("/streaming_jobs", get(list_streaming_jobs)) .route("/fragments/job_id/:job_id", get(list_fragments_by_job_id)) diff --git a/src/meta/src/manager/diagnose.rs b/src/meta/src/manager/diagnose.rs index f2d2cd58dd494..fd4d67c256685 100644 --- a/src/meta/src/manager/diagnose.rs +++ b/src/meta/src/manager/diagnose.rs @@ -70,8 +70,9 @@ impl DiagnoseCommand { let mut report = String::new(); let _ = writeln!( report, - "report created at: {}", - chrono::DateTime::::from(std::time::SystemTime::now()) + "report created at: {}\nversion: {}", + chrono::DateTime::::from(std::time::SystemTime::now()), + risingwave_common::current_cluster_version(), ); let _ = writeln!(report); self.write_catalog(&mut report).await;