From 5f7e5578920bb53238dbdfa65c29df4ff2b00c82 Mon Sep 17 00:00:00 2001 From: xfz <73645462+xuefengze@users.noreply.github.com> Date: Mon, 29 Jan 2024 15:36:21 +0800 Subject: [PATCH 01/14] fix(avro): correct parse `timestamp` data before Epoch (#14832) Co-authored-by: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> --- src/common/src/types/datetime.rs | 6 +++ src/connector/src/parser/avro/util.rs | 2 + src/connector/src/parser/unified/avro.rs | 53 ++++++++++++++---------- 3 files changed, 39 insertions(+), 22 deletions(-) diff --git a/src/common/src/types/datetime.rs b/src/common/src/types/datetime.rs index bc18858d678ec..af6b54b057c82 100644 --- a/src/common/src/types/datetime.rs +++ b/src/common/src/types/datetime.rs @@ -536,6 +536,12 @@ impl Timestamp { self.0.timestamp_nanos_opt().unwrap() } + pub fn with_millis(timestamp_millis: i64) -> Result { + let secs = timestamp_millis.div_euclid(1_000); + let nsecs = timestamp_millis.rem_euclid(1_000) * 1_000_000; + Self::with_secs_nsecs(secs, nsecs as u32) + } + pub fn with_micros(timestamp_micros: i64) -> Result { let secs = timestamp_micros.div_euclid(1_000_000); let nsecs = timestamp_micros.rem_euclid(1_000_000) * 1000; diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index 7cc9cf14c1f84..a6c5c6fbef5d1 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -103,6 +103,8 @@ fn avro_type_mapping(schema: &Schema) -> anyhow::Result { DataType::Decimal } Schema::Date => DataType::Date, + Schema::LocalTimestampMillis => DataType::Timestamp, + Schema::LocalTimestampMicros => DataType::Timestamp, Schema::TimestampMillis => DataType::Timestamptz, Schema::TimestampMicros => DataType::Timestamptz, Schema::Duration => DataType::Interval, diff --git a/src/connector/src/parser/unified/avro.rs b/src/connector/src/parser/unified/avro.rs index 02cdcb4de4ca5..af5658331270d 100644 --- a/src/connector/src/parser/unified/avro.rs +++ b/src/connector/src/parser/unified/avro.rs @@ -23,10 +23,11 @@ use chrono::Datelike; use itertools::Itertools; use num_bigint::{BigInt, Sign}; use risingwave_common::array::{ListValue, StructValue}; -use risingwave_common::cast::{i64_to_timestamp, i64_to_timestamptz}; use risingwave_common::error::Result as RwResult; use risingwave_common::log::LogSuppresser; -use risingwave_common::types::{DataType, Date, Datum, Interval, JsonbVal, ScalarImpl, Time}; +use risingwave_common::types::{ + DataType, Date, Datum, Interval, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, +}; use risingwave_common::util::iter_util::ZipEqFast; use super::{Access, AccessError, AccessResult}; @@ -181,19 +182,27 @@ impl<'a> AvroParseOptions<'a> { } (Some(DataType::Varchar) | None, Value::String(s)) => s.clone().into_boxed_str().into(), // ---- Timestamp ----- - (Some(DataType::Timestamp) | None, Value::TimestampMillis(ms)) => { - i64_to_timestamp(*ms).map_err(|_| create_error())?.into() + (Some(DataType::Timestamp) | None, Value::LocalTimestampMillis(ms)) => { + Timestamp::with_millis(*ms) + .map_err(|_| create_error())? + .into() } - (Some(DataType::Timestamp) | None, Value::TimestampMicros(us)) => { - i64_to_timestamp(*us).map_err(|_| create_error())?.into() + (Some(DataType::Timestamp) | None, Value::LocalTimestampMicros(us)) => { + Timestamp::with_micros(*us) + .map_err(|_| create_error())? + .into() } // ---- TimestampTz ----- - (Some(DataType::Timestamptz), Value::TimestampMillis(ms)) => { - i64_to_timestamptz(*ms).map_err(|_| create_error())?.into() + (Some(DataType::Timestamptz) | None, Value::TimestampMillis(ms)) => { + Timestamptz::from_millis(*ms) + .ok_or(AccessError::Other(anyhow!( + "timestamptz with milliseconds {ms} * 1000 is out of range", + )))? + .into() } - (Some(DataType::Timestamptz), Value::TimestampMicros(us)) => { - i64_to_timestamptz(*us).map_err(|_| create_error())?.into() + (Some(DataType::Timestamptz) | None, Value::TimestampMicros(us)) => { + Timestamptz::from_micros(*us).into() } // ---- Interval ----- @@ -424,7 +433,7 @@ pub(crate) fn unix_epoch_days() -> i32 { mod tests { use apache_avro::Decimal as AvroDecimal; use risingwave_common::error::{ErrorCode, RwError}; - use risingwave_common::types::{Decimal, Timestamp}; + use risingwave_common::types::{Decimal, Timestamptz}; use super::*; @@ -486,24 +495,24 @@ mod tests { } #[test] - fn test_avro_timestamp_micros() { - let v1 = Value::TimestampMicros(1620000000000); - let v2 = Value::TimestampMillis(1620000000); + fn test_avro_timestamptz_micros() { + let v1 = Value::TimestampMicros(1620000000000000); + let v2 = Value::TimestampMillis(1620000000000); let value_schema1 = Schema::TimestampMicros; let value_schema2 = Schema::TimestampMillis; - let datum1 = from_avro_value(v1, &value_schema1, &DataType::Timestamp).unwrap(); - let datum2 = from_avro_value(v2, &value_schema2, &DataType::Timestamp).unwrap(); + let datum1 = from_avro_value(v1, &value_schema1, &DataType::Timestamptz).unwrap(); + let datum2 = from_avro_value(v2, &value_schema2, &DataType::Timestamptz).unwrap(); assert_eq!( datum1, - Some(ScalarImpl::Timestamp(Timestamp::new( - "2021-05-03T00:00:00".parse().unwrap() - ))) + Some(ScalarImpl::Timestamptz( + Timestamptz::from_str("2021-05-03T00:00:00Z").unwrap() + )) ); assert_eq!( datum2, - Some(ScalarImpl::Timestamp(Timestamp::new( - "2021-05-03T00:00:00".parse().unwrap() - ))) + Some(ScalarImpl::Timestamptz( + Timestamptz::from_str("2021-05-03T00:00:00Z").unwrap() + )) ); } From e3e109a6a48ea054b0e5e39dd27d4f09f949d15f Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Mon, 29 Jan 2024 15:39:23 +0800 Subject: [PATCH 02/14] feat: support scaling down in recovery loop (#14825) Signed-off-by: Shanicky Chen --- risedev.yml | 22 +++++++++++++++ src/meta/src/barrier/recovery.rs | 33 +++++++++++++++++----- src/meta/src/manager/cluster.rs | 48 +++++++++++++++++++++----------- src/meta/src/stream/scale.rs | 2 +- 4 files changed, 80 insertions(+), 25 deletions(-) diff --git a/risedev.yml b/risedev.yml index 7be1334deb4b4..38ed00e15fc63 100644 --- a/risedev.yml +++ b/risedev.yml @@ -809,6 +809,28 @@ profile: - use: frontend - use: compactor + ci-3cn-1fe-with-recovery: + config-path: src/config/ci-recovery.toml + steps: + - use: minio + - use: etcd + unsafe-no-fsync: true + - use: meta-node + - use: compute-node + port: 5687 + exporter-port: 1222 + enable-tiered-cache: true + - use: compute-node + port: 5688 + exporter-port: 1223 + enable-tiered-cache: true + - use: compute-node + port: 5689 + exporter-port: 1224 + enable-tiered-cache: true + - use: frontend + - use: compactor + ci-1cn-1fe-kafka-with-recovery: config-path: src/config/ci-recovery.toml steps: diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index e0ace5f9678a4..c9e38bcad7d69 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -22,6 +22,7 @@ use futures::stream::FuturesUnordered; use futures::TryStreamExt; use itertools::Itertools; use risingwave_common::catalog::TableId; +use risingwave_common::hash::ParallelUnitId; use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_pb::common::ActorInfo; use risingwave_pb::meta::PausedReason; @@ -34,7 +35,7 @@ use risingwave_pb::stream_service::{ use thiserror_ext::AsReport; use tokio::sync::oneshot; use tokio_retry::strategy::{jitter, ExponentialBackoff}; -use tracing::{debug, warn, Instrument}; +use tracing::{debug, info, warn, Instrument}; use uuid::Uuid; use super::TracedEpoch; @@ -603,18 +604,36 @@ impl GlobalBarrierManagerContext { }; debug!("start scaling-in offline actors."); - let expired_workers: HashSet = info - .actor_map + let prev_worker_parallel_units = mgr.fragment_manager.all_worker_parallel_units().await; + + let curr_worker_parallel_units: HashMap> = info + .node_map .iter() - .filter(|(&worker, actors)| !actors.is_empty() && !info.node_map.contains_key(&worker)) - .map(|(&worker, _)| worker) + .map(|(worker_id, worker_node)| { + ( + *worker_id, + worker_node + .parallel_units + .iter() + .map(|parallel_unit| parallel_unit.id) + .collect(), + ) + }) .collect(); - if expired_workers.is_empty() { - debug!("no expired workers, skipping."); + // todo: maybe we can only check the reduced workers + if curr_worker_parallel_units == prev_worker_parallel_units { + debug!("no changed workers, skipping."); return Ok(false); } + info!("parallel unit has changed, triggering a forced reschedule."); + + debug!( + "previous worker parallel units {:?}, current worker parallel units {:?}", + prev_worker_parallel_units, curr_worker_parallel_units + ); + let table_parallelisms = { let guard = mgr.fragment_manager.get_fragment_read_guard().await; diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 8640088f30193..27a97167d3e4b 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -107,7 +107,7 @@ impl ClusterManager { property: AddNodeProperty, resource: risingwave_pb::common::worker_node::Resource, ) -> MetaResult { - let worker_node_parallelism = property.worker_node_parallelism as usize; + let new_worker_parallelism = property.worker_node_parallelism as usize; let mut property = self.parse_property(r#type, property); let mut core = self.core.write().await; @@ -123,8 +123,8 @@ impl ClusterManager { .unwrap_or_default(); } - let current_parallelism = worker.worker_node.parallel_units.len(); - if current_parallelism == worker_node_parallelism + let old_worker_parallelism = worker.worker_node.parallel_units.len(); + if old_worker_parallelism == new_worker_parallelism && worker.worker_node.property == property { worker.update_expire_at(self.max_heartbeat_interval); @@ -132,31 +132,45 @@ impl ClusterManager { } let mut new_worker = worker.clone(); - match current_parallelism.cmp(&worker_node_parallelism) { + match old_worker_parallelism.cmp(&new_worker_parallelism) { Ordering::Less => { tracing::info!( "worker {} parallelism updated from {} to {}", new_worker.worker_node.id, - current_parallelism, - worker_node_parallelism + old_worker_parallelism, + new_worker_parallelism ); let parallel_units = self .generate_cn_parallel_units( - worker_node_parallelism - current_parallelism, + new_worker_parallelism - old_worker_parallelism, new_worker.worker_id(), ) .await?; new_worker.worker_node.parallel_units.extend(parallel_units); } Ordering::Greater => { - // Warn and keep the original parallelism if the worker registered with a - // smaller parallelism. - tracing::warn!( - "worker {} parallelism is less than current, current is {}, but received {}", - new_worker.worker_id(), - current_parallelism, - worker_node_parallelism - ); + if self.env.opts.enable_scale_in_when_recovery { + // Handing over to the subsequent recovery loop for a forced reschedule. + tracing::info!( + "worker {} parallelism reduced from {} to {}", + new_worker.worker_node.id, + old_worker_parallelism, + new_worker_parallelism + ); + new_worker + .worker_node + .parallel_units + .truncate(new_worker_parallelism) + } else { + // Warn and keep the original parallelism if the worker registered with a + // smaller parallelism, entering compatibility mode. + tracing::warn!( + "worker {} parallelism is less than current, current is {}, but received {}", + new_worker.worker_id(), + new_worker_parallelism, + old_worker_parallelism, + ); + } } Ordering::Equal => {} } @@ -193,7 +207,7 @@ impl ClusterManager { // Generate parallel units. let parallel_units = if r#type == WorkerType::ComputeNode { - self.generate_cn_parallel_units(worker_node_parallelism, worker_id) + self.generate_cn_parallel_units(new_worker_parallelism, worker_id) .await? } else { vec![] @@ -550,7 +564,7 @@ impl ClusterManagerCore { None => { return Err(MetaError::unavailable( "no available transactional id for worker", - )) + )); } Some(id) => id, }; diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index 2674975488e45..e0a2fb390f42d 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -2420,7 +2420,7 @@ impl GlobalStreamManager { .prepare_reschedule_command(reschedules, options, table_parallelism.as_mut()) .await?; - tracing::debug!("reschedule plan: {:#?}", reschedule_fragment); + tracing::debug!("reschedule plan: {:?}", reschedule_fragment); let command = Command::RescheduleFragment { reschedules: reschedule_fragment, From 1a22e82cfcd414f9db0f9a3ce59cc4430f3ab015 Mon Sep 17 00:00:00 2001 From: "Zhanxiang (Patrick) Huang" Date: Mon, 29 Jan 2024 17:25:05 +0800 Subject: [PATCH 03/14] fix: enable compaction output check regardless of the ReportTask rpc status (#14840) --- src/storage/src/hummock/compactor/mod.rs | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index 158b5b33d414a..33710e95d47d9 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -502,18 +502,17 @@ pub fn start_compactor( .as_millis() as u64, }) { tracing::warn!(error = %e.as_report(), "Failed to report task {task_id:?}"); - if enable_check_compaction_result && need_check_task { - match check_compaction_result(&compact_task, context.clone()).await { - Err(e) => { - tracing::warn!(error = %e.as_report(), "Failed to check compaction task {}",compact_task.task_id); - }, - Ok(true) => (), - Ok(false) => { - panic!("Failed to pass consistency check for result of compaction task:\n{:?}", compact_task_to_string(&compact_task)); - } + } + if enable_check_compaction_result && need_check_task { + match check_compaction_result(&compact_task, context.clone()).await { + Err(e) => { + tracing::warn!(error = %e.as_report(), "Failed to check compaction task {}",compact_task.task_id); + }, + Ok(true) => (), + Ok(false) => { + panic!("Failed to pass consistency check for result of compaction task:\n{:?}", compact_task_to_string(&compact_task)); } } - } } ResponseEvent::VacuumTask(vacuum_task) => { From b471a9bd2d3da552055b7f449e8f13f6dd701e55 Mon Sep 17 00:00:00 2001 From: August Date: Mon, 29 Jan 2024 21:16:35 +0800 Subject: [PATCH 04/14] feat(sql-backend): support drop source/view/streaming jobs ddls in sql-backend (#14690) --- src/meta/src/barrier/command.rs | 64 +++++------ src/meta/src/barrier/recovery.rs | 59 ++++++---- src/meta/src/barrier/schedule.rs | 28 +++-- src/meta/src/controller/catalog.rs | 79 ++++++++++--- src/meta/src/controller/fragment.rs | 28 +---- src/meta/src/controller/streaming_job.rs | 61 ++++++---- src/meta/src/controller/utils.rs | 71 +++++++++++- src/meta/src/error.rs | 4 + src/meta/src/manager/metadata.rs | 28 ++--- src/meta/src/rpc/ddl_controller.rs | 114 +++++++++---------- src/meta/src/rpc/ddl_controller_v2.rs | 98 ++++++++++++++-- src/meta/src/stream/scale.rs | 32 +----- src/meta/src/stream/source_manager.rs | 13 ++- src/meta/src/stream/stream_manager.rs | 137 ++++++++++++----------- 14 files changed, 501 insertions(+), 315 deletions(-) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index de88c1ae17608..6ece437ffb2de 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -133,13 +133,13 @@ pub enum Command { Resume(PausedReason), /// `DropStreamingJobs` command generates a `Stop` barrier to stop the given - /// [`HashMap>`]. The catalog has ensured that these streaming jobs are safe to be + /// [`Vec`]. The catalog has ensured that these streaming jobs are safe to be /// dropped by reference counts before. /// /// Barriers from the actors to be dropped will STILL be collected. /// 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(HashMap>), + DropStreamingJobs(Vec), /// `CreateStreamingJob` command generates a `Add` barrier by given info. /// @@ -210,9 +210,9 @@ impl Command { Command::Plain(_) => None, Command::Pause(_) => None, Command::Resume(_) => None, - Command::DropStreamingJobs(node_actors) => Some(CommandActorChanges { + Command::DropStreamingJobs(actors) => Some(CommandActorChanges { to_add: Default::default(), - to_remove: node_actors.values().flatten().cloned().collect(), + to_remove: actors.iter().cloned().collect(), }), Command::CreateStreamingJob { table_fragments, @@ -397,10 +397,9 @@ impl CommandContext { })) } - Command::DropStreamingJobs(node_actors) => { - let actors = node_actors.values().flatten().copied().collect(); - Some(Mutation::Stop(StopMutation { actors })) - } + Command::DropStreamingJobs(actors) => Some(Mutation::Stop(StopMutation { + actors: actors.clone(), + })), Command::CreateStreamingJob { table_fragments, @@ -740,13 +739,10 @@ impl CommandContext { } /// Clean up actors in CNs if needed, used by drop, cancel and reschedule commands. - async fn clean_up( - &self, - actors_to_clean: impl IntoIterator)>, - ) -> MetaResult<()> { - let futures = actors_to_clean.into_iter().map(|(node_id, actors)| { - let node = self.info.node_map.get(&node_id).unwrap(); + async fn clean_up(&self, actors: Vec) -> MetaResult<()> { + let futures = self.info.node_map.values().map(|node| { let request_id = Uuid::new_v4().to_string(); + let actor_ids = actors.clone(); async move { let client = self @@ -757,7 +753,7 @@ impl CommandContext { .await?; let request = DropActorsRequest { request_id, - actor_ids: actors.to_owned(), + actor_ids, }; client.drop_actors(request).await } @@ -815,15 +811,14 @@ impl CommandContext { .await; } - Command::DropStreamingJobs(node_actors) => { + Command::DropStreamingJobs(actors) => { // Tell compute nodes to drop actors. - self.clean_up(node_actors.clone()).await?; + self.clean_up(actors.clone()).await?; } Command::CancelStreamingJob(table_fragments) => { tracing::debug!(id = ?table_fragments.table_id(), "cancelling stream job"); - let node_actors = table_fragments.worker_actor_ids(); - self.clean_up(node_actors).await?; + self.clean_up(table_fragments.actor_ids()).await?; // NOTE(kwannoel): At this point, meta has already registered the table ids. // We should unregister them. @@ -878,8 +873,10 @@ impl CommandContext { ))) .await?; } - MetadataManager::V2(_mgr) => { - unimplemented!("implement cancel for sql backend") + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .try_abort_creating_streaming_job(table_id as _, true) + .await?; } } } @@ -921,12 +918,7 @@ impl CommandContext { init_split_assignment, }) = replace_table { - let table_ids = - HashSet::from_iter(std::iter::once(old_table_fragments.table_id())); - // Tell compute nodes to drop actors. - let node_actors = - mgr.fragment_manager.table_node_actors(&table_ids).await?; - self.clean_up(node_actors).await?; + self.clean_up(old_table_fragments.actor_ids()).await?; // Drop fragment info in meta store. mgr.fragment_manager @@ -969,14 +961,17 @@ impl CommandContext { reschedules, table_parallelism, } => { - let node_dropped_actors = self - .barrier_manager_context + let removed_actors = reschedules + .values() + .flat_map(|reschedule| reschedule.removed_actors.clone().into_iter()) + .collect_vec(); + self.clean_up(removed_actors).await?; + self.barrier_manager_context .scale_controller .as_ref() .unwrap() .post_apply_reschedule(reschedules, table_parallelism) .await?; - self.clean_up(node_dropped_actors).await?; } Command::ReplaceTable(ReplaceTablePlan { @@ -986,14 +981,7 @@ impl CommandContext { dispatchers, init_split_assignment, }) => { - let table_ids = HashSet::from_iter(std::iter::once(old_table_fragments.table_id())); - // Tell compute nodes to drop actors. - let node_actors = self - .barrier_manager_context - .metadata_manager - .get_worker_actor_ids(table_ids) - .await?; - self.clean_up(node_actors).await?; + self.clean_up(old_table_fragments.actor_ids()).await?; match &self.barrier_manager_context.metadata_manager { MetadataManager::V1(mgr) => { diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index c9e38bcad7d69..12e298648f5db 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -138,9 +138,7 @@ impl GlobalBarrierManagerContext { } async fn recover_background_mv_progress_v1(&self) -> MetaResult<()> { - let MetadataManager::V1(mgr) = &self.metadata_manager else { - unreachable!() - }; + let mgr = self.metadata_manager.as_v1_ref(); let mviews = mgr.catalog_manager.list_creating_background_mvs().await; let mut mview_definitions = HashMap::new(); @@ -228,9 +226,7 @@ impl GlobalBarrierManagerContext { } async fn recover_background_mv_progress_v2(&self) -> MetaResult<()> { - let MetadataManager::V2(mgr) = &self.metadata_manager else { - unreachable!() - }; + let mgr = self.metadata_manager.as_v2_ref(); let mviews = mgr .catalog_controller .list_background_creating_mviews() @@ -306,6 +302,33 @@ impl GlobalBarrierManagerContext { Ok(()) } + /// Pre buffered drop and cancel command, return true if any. + async fn pre_apply_drop_cancel( + &self, + scheduled_barriers: &ScheduledBarriers, + ) -> MetaResult { + let (dropped_actors, cancelled) = + scheduled_barriers.pre_apply_drop_cancel_scheduled().await; + let applied = !dropped_actors.is_empty() || !cancelled.is_empty(); + if !cancelled.is_empty() { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.fragment_manager + .drop_table_fragments_vec(&cancelled) + .await?; + } + MetadataManager::V2(mgr) => { + for job_id in cancelled { + mgr.catalog_controller + .try_abort_creating_streaming_job(job_id.table_id as _, true) + .await?; + } + } + } + } + Ok(applied) + } + /// Recovery the whole cluster from the latest epoch. /// /// If `paused_reason` is `Some`, all data sources (including connectors and DMLs) will be @@ -347,7 +370,7 @@ impl GlobalBarrierManagerContext { async { let recovery_result: MetaResult<_> = try { // This is a quick path to accelerate the process of dropping and canceling streaming jobs. - let _ = scheduled_barriers.pre_apply_drop_scheduled().await; + let _ = self.pre_apply_drop_cancel(scheduled_barriers).await?; // Resolve actor info for recovery. If there's no actor to recover, most of the // following steps will be no-op, while the compute nodes will still be reset. @@ -373,7 +396,7 @@ impl GlobalBarrierManagerContext { warn!(error = %err.as_report(), "reset compute nodes failed"); })?; - if scheduled_barriers.pre_apply_drop_scheduled().await { + if self.pre_apply_drop_cancel(scheduled_barriers).await? { info = self.resolve_actor_info().await; } @@ -470,9 +493,7 @@ impl GlobalBarrierManagerContext { } async fn migrate_actors_v2(&self) -> MetaResult { - let MetadataManager::V2(mgr) = &self.metadata_manager else { - unreachable!() - }; + let mgr = self.metadata_manager.as_v2_ref(); let all_inuse_parallel_units: HashSet<_> = mgr .catalog_controller @@ -552,9 +573,7 @@ impl GlobalBarrierManagerContext { /// Migrate actors in expired CNs to newly joined ones, return true if any actor is migrated. async fn migrate_actors_v1(&self) -> MetaResult { - let MetadataManager::V1(mgr) = &self.metadata_manager else { - unreachable!() - }; + let mgr = self.metadata_manager.as_v1_ref(); let info = self.resolve_actor_info().await; @@ -592,16 +611,12 @@ impl GlobalBarrierManagerContext { } fn scale_actors_v2(&self, _info: &InflightActorInfo) -> MetaResult { - let MetadataManager::V2(_mgr) = &self.metadata_manager else { - unreachable!() - }; + let _mgr = self.metadata_manager.as_v2_ref(); unimplemented!("implement auto-scale funcs in sql backend") } async fn scale_actors_v1(&self, info: &InflightActorInfo) -> MetaResult { - let MetadataManager::V1(mgr) = &self.metadata_manager else { - unreachable!() - }; + let mgr = self.metadata_manager.as_v1_ref(); debug!("start scaling-in offline actors."); let prev_worker_parallel_units = mgr.fragment_manager.all_worker_parallel_units().await; @@ -713,9 +728,7 @@ impl GlobalBarrierManagerContext { &self, expired_workers: HashSet, ) -> MetaResult { - let MetadataManager::V1(mgr) = &self.metadata_manager else { - unreachable!() - }; + let mgr = self.metadata_manager.as_v1_ref(); let mut cached_plan = MigrationPlan::get(self.env.meta_store()).await?; diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index 26fd3ea8143ef..56152c18baa70 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::VecDeque; +use std::collections::{HashSet, VecDeque}; use std::iter::once; use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; use std::sync::Arc; @@ -28,6 +28,7 @@ use tokio::sync::{oneshot, watch, RwLock}; use super::notifier::{BarrierInfo, Notifier}; use super::{Command, Scheduled}; use crate::hummock::HummockManagerRef; +use crate::model::ActorId; use crate::rpc::metrics::MetaMetrics; use crate::{MetaError, MetaResult}; @@ -392,28 +393,35 @@ impl ScheduledBarriers { queue.mark_ready(); } - /// Try to pre apply drop scheduled command and return true if any. + /// Try to pre apply drop and cancel scheduled command and return them if any. /// It should only be called in recovery. - pub(super) async fn pre_apply_drop_scheduled(&self) -> bool { + pub(super) async fn pre_apply_drop_cancel_scheduled(&self) -> (Vec, HashSet) { let mut queue = self.inner.queue.write().await; assert_matches!(queue.status, QueueStatus::Blocked(_)); - let mut found = false; + let (mut drop_table_ids, mut cancel_table_ids) = (vec![], HashSet::new()); while let Some(Scheduled { notifiers, command, .. }) = queue.queue.pop_front() { - assert_matches!( - command, - Command::DropStreamingJobs(_) | Command::CancelStreamingJob(_) - ); + match command { + Command::DropStreamingJobs(actor_ids) => { + drop_table_ids.extend(actor_ids); + } + Command::CancelStreamingJob(table_fragments) => { + let table_id = table_fragments.table_id(); + cancel_table_ids.insert(table_id); + } + _ => { + unreachable!("only drop and cancel streaming jobs should be buffered"); + } + } notifiers.into_iter().for_each(|mut notify| { notify.notify_collected(); notify.notify_finished(); }); - found = true; } - found + (drop_table_ids, cancel_table_ids) } /// Whether the barrier(checkpoint = true) should be injected. diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 70fc1c52a0fce..d4f67c6ad1ee9 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{BTreeSet, HashMap, HashSet}; use std::iter; use std::sync::Arc; @@ -25,9 +25,9 @@ use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::table::TableType; use risingwave_meta_model_v2::{ connection, database, function, index, object, object_dependency, schema, sink, source, - streaming_job, table, user_privilege, view, ColumnCatalogArray, ConnectionId, CreateType, - DatabaseId, FunctionId, IndexId, JobStatus, ObjectId, PrivateLinkService, Property, SchemaId, - SourceId, StreamSourceInfo, TableId, UserId, + streaming_job, table, user_privilege, view, ActorId, ColumnCatalogArray, ConnectionId, + CreateType, DatabaseId, FragmentId, FunctionId, IndexId, JobStatus, ObjectId, + PrivateLinkService, Property, SchemaId, SourceId, StreamSourceInfo, TableId, UserId, }; use risingwave_pb::catalog::table::PbTableType; use risingwave_pb::catalog::{ @@ -56,7 +56,8 @@ use crate::controller::utils::{ check_function_signature_duplicate, check_relation_name_duplicate, check_schema_name_duplicate, ensure_object_id, ensure_object_not_refer, ensure_schema_empty, ensure_user_id, get_fragment_mappings, get_referring_objects, get_referring_objects_cascade, - get_user_privilege, list_user_info_by_ids, PartialObject, + get_user_privilege, list_user_info_by_ids, resolve_source_register_info_for_jobs, + PartialObject, }; use crate::controller::ObjectModel; use crate::manager::{Catalog, MetaSrvEnv, NotificationVersion, IGNORED_NOTIFICATION_VERSION}; @@ -74,11 +75,17 @@ pub struct CatalogController { #[derive(Clone, Default)] pub struct ReleaseContext { - pub(crate) streaming_jobs: Vec, - #[allow(dead_code)] + /// Dropped state table list, need to unregister from hummock. pub(crate) state_table_ids: Vec, + /// Dropped source list, need to unregister from source manager. pub(crate) source_ids: Vec, + /// Dropped connection list, need to delete from vpc endpoints. pub(crate) connections: Vec, + + /// Dropped fragments that are fetching data from the target source. + pub(crate) source_fragments: HashMap>, + /// Dropped actors. + pub(crate) removed_actors: HashSet, } impl CatalogController { @@ -200,18 +207,18 @@ impl CatalogController { let txn = inner.db.begin().await?; ensure_object_id(ObjectType::Database, database_id, &txn).await?; - let streaming_jobs: Vec = Object::find() + let streaming_jobs: Vec = StreamingJob::find() + .join(JoinType::InnerJoin, streaming_job::Relation::Object.def()) .select_only() - .column(object::Column::Oid) - .filter( - object::Column::DatabaseId - .eq(Some(database_id)) - .and(object::Column::ObjType.is_in([ObjectType::Table, ObjectType::Sink])), - ) + .column(streaming_job::Column::JobId) + .filter(object::Column::DatabaseId.eq(Some(database_id))) .into_tuple() .all(&txn) .await?; + let (source_fragments, removed_actors) = + resolve_source_register_info_for_jobs(&txn, streaming_jobs.clone()).await?; + let state_table_ids: Vec = Table::find() .select_only() .column(table::Column::TableId) @@ -281,10 +288,11 @@ impl CatalogController { .await; Ok(( ReleaseContext { - streaming_jobs, state_table_ids, source_ids, connections, + source_fragments, + removed_actors, }, version, )) @@ -452,15 +460,20 @@ impl CatalogController { .all(&txn) .await?; + let to_delete_objs: HashSet = creating_job_ids + .clone() + .into_iter() + .chain(state_table_ids.clone().into_iter()) + .collect(); + let res = Object::delete_many() - .filter(object::Column::Oid.is_in(creating_job_ids.clone())) + .filter(object::Column::Oid.is_in(to_delete_objs)) .exec(&txn) .await?; assert!(res.rows_affected > 0); txn.commit().await?; Ok(ReleaseContext { - streaming_jobs: creating_job_ids, state_table_ids, source_ids: associated_source_ids, ..Default::default() @@ -580,6 +593,18 @@ impl CatalogController { )), }); } + ObjectType::Source => { + let (source, obj) = Source::find_by_id(job_id) + .find_also_related(Object) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("source", job_id))?; + relations.push(PbRelation { + relation_info: Some(PbRelationInfo::Source( + ObjectModel(source, obj.unwrap()).into(), + )), + }); + } _ => unreachable!("invalid job type: {:?}", job_type), } @@ -1438,6 +1463,20 @@ impl CatalogController { } } + let creating = StreamingJob::find() + .filter( + streaming_job::Column::JobStatus + .ne(JobStatus::Created) + .and(streaming_job::Column::JobId.is_in(to_drop_streaming_jobs.clone())), + ) + .count(&txn) + .await?; + if creating != 0 { + return Err(MetaError::permission_denied(format!( + "can not drop {creating} creating streaming job, please cancel them firstly" + ))); + } + let mut to_drop_state_table_ids = to_drop_table_ids.clone().collect_vec(); let to_drop_index_ids = to_drop_objects .iter() @@ -1496,6 +1535,9 @@ impl CatalogController { to_drop_objects.extend(to_drop_internal_table_objs); } + let (source_fragments, removed_actors) = + resolve_source_register_info_for_jobs(&txn, to_drop_streaming_jobs).await?; + // Find affect users with privileges on all this objects. let to_update_user_ids: Vec = UserPrivilege::find() .select_only() @@ -1578,10 +1620,11 @@ impl CatalogController { Ok(( ReleaseContext { - streaming_jobs: to_drop_streaming_jobs, state_table_ids: to_drop_state_table_ids, source_ids: to_drop_source_ids, connections: vec![], + source_fragments, + removed_actors, }, version, )) diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 0de826fef9f86..d5e50c6fecfe1 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -39,8 +39,7 @@ use risingwave_pb::meta::{ use risingwave_pb::source::PbConnectorSplits; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ - PbDispatchStrategy, PbFragmentTypeFlag, PbStreamActor, PbStreamContext, PbStreamNode, - StreamSource, + PbDispatchStrategy, PbFragmentTypeFlag, PbStreamActor, PbStreamContext, }; use sea_orm::sea_query::{Expr, Value}; use sea_orm::ActiveValue::Set; @@ -51,8 +50,8 @@ use sea_orm::{ use crate::controller::catalog::{CatalogController, CatalogControllerInner}; use crate::controller::utils::{ - get_actor_dispatchers, get_parallel_unit_mapping, FragmentDesc, PartialActorLocation, - PartialFragmentStateTables, + find_stream_source, get_actor_dispatchers, get_parallel_unit_mapping, FragmentDesc, + PartialActorLocation, PartialFragmentStateTables, }; use crate::manager::{ActorInfos, LocalNotification}; use crate::stream::SplitAssignment; @@ -1160,23 +1159,6 @@ impl CatalogController { Ok(chain_fragments) } - /// Find the external stream source info inside the stream node, if any. - fn find_stream_source(stream_node: &PbStreamNode) -> Option<&StreamSource> { - if let Some(NodeBody::Source(source)) = &stream_node.node_body { - if let Some(inner) = &source.source_inner { - return Some(inner); - } - } - - for child in &stream_node.input { - if let Some(source) = Self::find_stream_source(child) { - return Some(source); - } - } - - None - } - pub async fn load_source_fragment_ids( &self, ) -> MetaResult>> { @@ -1195,7 +1177,7 @@ impl CatalogController { let mut source_fragment_ids = HashMap::new(); for (fragment_id, _, stream_node) in fragments { - if let Some(source) = Self::find_stream_source(stream_node.inner_ref()) { + if let Some(source) = find_stream_source(stream_node.inner_ref()) { source_fragment_ids .entry(source.source_id as SourceId) .or_insert_with(BTreeSet::new) @@ -1225,7 +1207,7 @@ impl CatalogController { let mut source_fragment_ids = HashMap::new(); for (fragment_id, _, stream_node) in fragments { - if let Some(source) = Self::find_stream_source(stream_node.inner_ref()) { + if let Some(source) = find_stream_source(stream_node.inner_ref()) { source_fragment_ids .entry(source.source_id as SourceId) .or_insert_with(BTreeSet::new) diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index 576c5c3c40699..c7547632e6105 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -23,10 +23,10 @@ use risingwave_meta_model_v2::prelude::{ Actor, ActorDispatcher, Fragment, Index, Object, ObjectDependency, Source, Table, }; use risingwave_meta_model_v2::{ - actor, actor_dispatcher, fragment, index, object_dependency, sink, source, streaming_job, - table, ActorId, ActorUpstreamActors, CreateType, DatabaseId, ExprNodeArray, FragmentId, - I32Array, IndexId, JobStatus, ObjectId, SchemaId, SourceId, StreamNode, TableId, TableVersion, - UserId, + actor, actor_dispatcher, fragment, index, object, object_dependency, sink, source, + streaming_job, table, ActorId, ActorUpstreamActors, CreateType, DatabaseId, ExprNodeArray, + FragmentId, I32Array, IndexId, JobStatus, ObjectId, SchemaId, SourceId, StreamNode, TableId, + TableVersion, UserId, }; use risingwave_pb::catalog::source::PbOptionalAssociatedTableId; use risingwave_pb::catalog::table::{PbOptionalAssociatedSourceId, PbTableVersion}; @@ -44,7 +44,7 @@ use sea_orm::sea_query::SimpleExpr; use sea_orm::ActiveValue::Set; use sea_orm::{ ActiveEnum, ActiveModelTrait, ColumnTrait, DatabaseTransaction, EntityTrait, IntoActiveModel, - ModelTrait, NotSet, QueryFilter, QuerySelect, TransactionTrait, + ModelTrait, NotSet, PaginatorTrait, QueryFilter, QuerySelect, TransactionTrait, }; use crate::controller::catalog::CatalogController; @@ -321,32 +321,55 @@ impl CatalogController { /// `try_abort_creating_streaming_job` is used to abort the job that is under initial status or in `FOREGROUND` mode. /// It returns true if the job is not found or aborted. - pub async fn try_abort_creating_streaming_job(&self, job_id: ObjectId) -> MetaResult { + pub async fn try_abort_creating_streaming_job( + &self, + job_id: ObjectId, + is_cancelled: bool, + ) -> MetaResult { let inner = self.inner.write().await; let txn = inner.db.begin().await?; - let streaming_job = streaming_job::Entity::find_by_id(job_id).one(&txn).await?; - let Some(streaming_job) = streaming_job else { + let cnt = Object::find_by_id(job_id).count(&txn).await?; + if cnt == 0 { tracing::warn!( id = job_id, "streaming job not found when aborting creating, might be cleaned by recovery" ); return Ok(true); - }; + } - assert_ne!(streaming_job.job_status, JobStatus::Created); - if streaming_job.create_type == CreateType::Background - && streaming_job.job_status == JobStatus::Creating - { - // If the job is created in background and still in creating status, we should not abort it and let recovery to handle it. - tracing::warn!( - id = job_id, - "streaming job is created in background and still in creating status" - ); - return Ok(false); + if !is_cancelled { + let streaming_job = streaming_job::Entity::find_by_id(job_id).one(&txn).await?; + if let Some(streaming_job) = streaming_job { + assert_ne!(streaming_job.job_status, JobStatus::Created); + if streaming_job.create_type == CreateType::Background + && streaming_job.job_status == JobStatus::Creating + { + // If the job is created in background and still in creating status, we should not abort it and let recovery to handle it. + tracing::warn!( + id = job_id, + "streaming job is created in background and still in creating status" + ); + return Ok(false); + } + } } + let internal_table_ids: Vec = Table::find() + .select_only() + .column(table::Column::TableId) + .filter(table::Column::BelongsToJobId.eq(job_id)) + .into_tuple() + .all(&txn) + .await?; + Object::delete_by_id(job_id).exec(&txn).await?; + if !internal_table_ids.is_empty() { + Object::delete_many() + .filter(object::Column::Oid.is_in(internal_table_ids)) + .exec(&txn) + .await?; + } txn.commit().await?; Ok(true) diff --git a/src/meta/src/controller/utils.rs b/src/meta/src/controller/utils.rs index 76364697195f2..893f8812a86d3 100644 --- a/src/meta/src/controller/utils.rs +++ b/src/meta/src/controller/utils.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{BTreeSet, HashMap, HashSet}; use anyhow::anyhow; use itertools::Itertools; @@ -25,11 +25,13 @@ use risingwave_meta_model_v2::{ actor, actor_dispatcher, connection, database, fragment, function, index, object, object_dependency, schema, sink, source, table, user, user_privilege, view, worker_property, ActorId, DataTypeArray, DatabaseId, FragmentId, FragmentVnodeMapping, I32Array, ObjectId, - PrivilegeId, SchemaId, UserId, WorkerId, + PrivilegeId, SchemaId, SourceId, StreamNode, UserId, WorkerId, }; use risingwave_pb::catalog::{PbConnection, PbFunction}; use risingwave_pb::common::PbParallelUnit; use risingwave_pb::meta::PbFragmentParallelUnitMapping; +use risingwave_pb::stream_plan::stream_node::NodeBody; +use risingwave_pb::stream_plan::{PbFragmentTypeFlag, PbStreamNode, StreamSource}; use risingwave_pb::user::grant_privilege::{PbAction, PbActionWithGrantOption, PbObject}; use risingwave_pb::user::{PbGrantPrivilege, PbUserInfo}; use sea_orm::sea_query::{ @@ -710,3 +712,68 @@ where Ok(fragment_actors.into_iter().into_group_map()) } + +/// Find the external stream source info inside the stream node, if any. +pub fn find_stream_source(stream_node: &PbStreamNode) -> Option<&StreamSource> { + if let Some(NodeBody::Source(source)) = &stream_node.node_body { + if let Some(inner) = &source.source_inner { + return Some(inner); + } + } + + for child in &stream_node.input { + if let Some(source) = find_stream_source(child) { + return Some(source); + } + } + + None +} + +/// Resolve fragment list that are subscribing to sources and actor lists. +pub async fn resolve_source_register_info_for_jobs( + db: &C, + streaming_jobs: Vec, +) -> MetaResult<(HashMap>, HashSet)> +where + C: ConnectionTrait, +{ + if streaming_jobs.is_empty() { + return Ok((HashMap::default(), HashSet::default())); + } + + let mut fragments: Vec<(FragmentId, i32, StreamNode)> = Fragment::find() + .select_only() + .columns([ + fragment::Column::FragmentId, + fragment::Column::FragmentTypeMask, + fragment::Column::StreamNode, + ]) + .filter(fragment::Column::JobId.is_in(streaming_jobs)) + .into_tuple() + .all(db) + .await?; + let actors: Vec = Actor::find() + .select_only() + .column(actor::Column::ActorId) + .filter( + actor::Column::FragmentId.is_in(fragments.iter().map(|(id, _, _)| *id).collect_vec()), + ) + .into_tuple() + .all(db) + .await?; + + fragments.retain(|(_, mask, _)| *mask & PbFragmentTypeFlag::Source as i32 != 0); + + let mut source_fragment_ids = HashMap::new(); + for (fragment_id, _, stream_node) in fragments { + if let Some(source) = find_stream_source(stream_node.inner_ref()) { + source_fragment_ids + .entry(source.source_id as SourceId) + .or_insert_with(BTreeSet::new) + .insert(fragment_id); + } + } + + Ok((source_fragment_ids, actors.into_iter().collect())) +} diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index 899fc3f13bee0..9833a51cc1934 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -119,6 +119,10 @@ impl MetaError { matches!(self.inner(), MetaErrorInner::FragmentNotFound(..)) } + pub fn is_cancelled(&self) -> bool { + matches!(self.inner(), MetaErrorInner::Cancelled(..)) + } + pub fn catalog_duplicated>(relation: &'static str, name: T) -> Self { MetaErrorInner::Duplicated(relation, name.into()).into() } diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index d4ed2e0e15577..4606edfb1dd1e 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -74,6 +74,20 @@ impl MetadataManager { }) } + pub fn as_v1_ref(&self) -> &MetadataManagerV1 { + match self { + MetadataManager::V1(mgr) => mgr, + MetadataManager::V2(_) => panic!("expect v1, found v2"), + } + } + + pub fn as_v2_ref(&self) -> &MetadataManagerV2 { + match self { + MetadataManager::V1(_) => panic!("expect v2, found v1"), + MetadataManager::V2(mgr) => mgr, + } + } + pub async fn get_worker_by_id(&self, worker_id: WorkerId) -> MetaResult> { match &self { MetadataManager::V1(mgr) => Ok(mgr @@ -419,20 +433,6 @@ impl MetadataManager { } } - pub async fn drop_streaming_job_by_ids(&self, table_ids: &HashSet) -> MetaResult<()> { - match self { - MetadataManager::V1(mgr) => { - mgr.fragment_manager - .drop_table_fragments_vec(table_ids) - .await - } - MetadataManager::V2(_) => { - // Do nothing. Need to refine drop and cancel process. - Ok(()) - } - } - } - pub async fn update_source_rate_limit_by_source_id( &self, source_id: SourceId, diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index e4296f7f403c2..55f86f93ea747 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -64,7 +64,6 @@ use tracing::log::warn; use tracing::Instrument; use crate::barrier::BarrierManagerRef; -use crate::controller::catalog::{CatalogControllerRef, ReleaseContext}; use crate::manager::{ CatalogManagerRef, ConnectionId, DatabaseId, FragmentManagerRef, FunctionId, IdCategory, IdCategoryType, IndexId, LocalNotification, MetaSrvEnv, MetadataManager, MetadataManagerV1, @@ -382,45 +381,14 @@ impl DdlController { Ok(version) } - async fn drop_database_v2( - &self, - catalog_controller: &CatalogControllerRef, - database_id: DatabaseId, - ) -> MetaResult { - let ( - ReleaseContext { - streaming_jobs, - source_ids, - connections, - .. - }, - version, - ) = catalog_controller.drop_database(database_id as _).await?; - self.source_manager - .unregister_sources(source_ids.into_iter().map(|id| id as _).collect()) - .await; - self.stream_manager - .drop_streaming_jobs( - streaming_jobs - .into_iter() - .map(|id| (id as u32).into()) - .collect(), - ) - .await; - for svc in connections { - self.delete_vpc_endpoint_v2(svc.into_inner()).await?; - } - Ok(version) - } - async fn drop_database(&self, database_id: DatabaseId) -> MetaResult { match &self.metadata_manager { MetadataManager::V1(mgr) => { self.drop_database_v1(&mgr.catalog_manager, database_id) .await } - MetadataManager::V2(mgr) => { - self.drop_database_v2(&mgr.catalog_controller, database_id) + MetadataManager::V2(_) => { + self.drop_object(ObjectType::Database, database_id as _, DropMode::Cascade) .await } } @@ -439,9 +407,8 @@ impl DdlController { async fn drop_schema(&self, schema_id: SchemaId) -> MetaResult { match &self.metadata_manager { MetadataManager::V1(mgr) => mgr.catalog_manager.drop_schema(schema_id).await, - MetadataManager::V2(mgr) => { - mgr.catalog_controller - .drop_schema(schema_id as _, DropMode::Restrict) + MetadataManager::V2(_) => { + self.drop_object(ObjectType::Schema, schema_id as _, DropMode::Restrict) .await } } @@ -484,7 +451,9 @@ impl DdlController { drop_mode: DropMode, ) -> MetaResult { let MetadataManager::V1(mgr) = &self.metadata_manager else { - unimplemented!("support drop source in v2"); + return self + .drop_object(ObjectType::Source, source_id as _, drop_mode) + .await; }; // 1. Drop source in catalog. let (version, streaming_job_ids) = mgr @@ -552,7 +521,9 @@ impl DdlController { drop_mode: DropMode, ) -> MetaResult { let MetadataManager::V1(mgr) = &self.metadata_manager else { - unimplemented!("support drop view in v2"); + return self + .drop_object(ObjectType::View, view_id as _, drop_mode) + .await; }; let (version, streaming_job_ids) = mgr .catalog_manager @@ -585,20 +556,25 @@ impl DdlController { &self, connection_id: ConnectionId, ) -> MetaResult { - let (version, connection) = match &self.metadata_manager { - MetadataManager::V1(mgr) => mgr.catalog_manager.drop_connection(connection_id).await?, - MetadataManager::V2(mgr) => { - mgr.catalog_controller - .drop_connection(connection_id as _) - .await? + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let (version, connection) = + mgr.catalog_manager.drop_connection(connection_id).await?; + self.delete_vpc_endpoint(&connection).await?; + Ok(version) } - }; - - self.delete_vpc_endpoint(&connection).await?; - Ok(version) + MetadataManager::V2(_) => { + self.drop_object( + ObjectType::Connection, + connection_id as _, + DropMode::Restrict, + ) + .await + } + } } - async fn delete_vpc_endpoint(&self, connection: &Connection) -> MetaResult<()> { + pub(crate) async fn delete_vpc_endpoint(&self, connection: &Connection) -> MetaResult<()> { // delete AWS vpc endpoint if let Some(connection::Info::PrivateLinkService(svc)) = &connection.info && svc.get_provider()? == PbPrivateLinkProvider::Aws @@ -615,7 +591,7 @@ impl DdlController { Ok(()) } - async fn delete_vpc_endpoint_v2(&self, svc: PrivateLinkService) -> MetaResult<()> { + pub(crate) async fn delete_vpc_endpoint_v2(&self, svc: PrivateLinkService) -> MetaResult<()> { // delete AWS vpc endpoint if svc.get_provider()? == PbPrivateLinkProvider::Aws { if let Some(aws_cli) = self.aws_client.as_ref() { @@ -1136,9 +1112,33 @@ impl DdlController { drop_mode: DropMode, target_replace_info: Option, ) -> MetaResult { - let MetadataManager::V1(mgr) = &self.metadata_manager else { - unimplemented!("support drop streaming job in v2"); - }; + match &self.metadata_manager { + MetadataManager::V1(_) => { + self.drop_streaming_job_v1(job_id, drop_mode, target_replace_info) + .await + } + MetadataManager::V2(_) => { + if target_replace_info.is_some() { + unimplemented!("support replace table for drop in v2"); + } + let (object_id, object_type) = match job_id { + StreamingJobId::MaterializedView(id) => (id as _, ObjectType::Table), + StreamingJobId::Sink(id) => (id as _, ObjectType::Sink), + StreamingJobId::Table(_, id) => (id as _, ObjectType::Table), + StreamingJobId::Index(idx) => (idx as _, ObjectType::Index), + }; + self.drop_object(object_type, object_id, drop_mode).await + } + } + } + + async fn drop_streaming_job_v1( + &self, + job_id: StreamingJobId, + drop_mode: DropMode, + target_replace_info: Option, + ) -> MetaResult { + let mgr = self.metadata_manager.as_v1_ref(); let _reschedule_job_lock = self.stream_manager.reschedule_lock.read().await; let (mut version, streaming_job_ids) = match job_id { StreamingJobId::MaterializedView(table_id) => { @@ -1347,7 +1347,7 @@ impl DdlController { bail!("additional replace table event only occurs when sinking into table"); }; let MetadataManager::V1(mgr) = &self.metadata_manager else { - unimplemented!("support replace table in v2"); + unimplemented!("support create sink into table in v2"); }; Some( @@ -1403,9 +1403,7 @@ impl DdlController { internal_tables: Vec, error: Option<&impl ToString>, ) -> MetaResult<()> { - let MetadataManager::V1(mgr) = &self.metadata_manager else { - unimplemented!("support cancel streaming job in v2"); - }; + let mgr = self.metadata_manager.as_v1_ref(); let error = error.map(ToString::to_string).unwrap_or_default(); let event = risingwave_pb::meta::event_log::EventCreateStreamJobFail { id: stream_job.id(), diff --git a/src/meta/src/rpc/ddl_controller_v2.rs b/src/meta/src/rpc/ddl_controller_v2.rs index fc620f6386a54..946f16078bcf5 100644 --- a/src/meta/src/rpc/ddl_controller_v2.rs +++ b/src/meta/src/rpc/ddl_controller_v2.rs @@ -15,6 +15,8 @@ use itertools::Itertools; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::stream_graph_visitor::visit_fragment; +use risingwave_meta_model_v2::object::ObjectType; +use risingwave_meta_model_v2::ObjectId; use risingwave_pb::catalog::CreateType; use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::stream_plan::stream_node::NodeBody; @@ -22,12 +24,12 @@ use risingwave_pb::stream_plan::update_mutation::PbMergeUpdate; use risingwave_pb::stream_plan::StreamFragmentGraph as StreamFragmentGraphProto; use thiserror_ext::AsReport; +use crate::controller::catalog::ReleaseContext; use crate::manager::{ - MetadataManager, MetadataManagerV2, NotificationVersion, StreamingJob, - IGNORED_NOTIFICATION_VERSION, + MetadataManagerV2, NotificationVersion, StreamingJob, IGNORED_NOTIFICATION_VERSION, }; use crate::model::{MetadataModel, StreamContext}; -use crate::rpc::ddl_controller::{fill_table_stream_graph_info, DdlController}; +use crate::rpc::ddl_controller::{fill_table_stream_graph_info, DdlController, DropMode}; use crate::stream::{validate_sink, StreamFragmentGraph}; use crate::MetaResult; @@ -37,9 +39,7 @@ impl DdlController { mut streaming_job: StreamingJob, mut fragment_graph: StreamFragmentGraphProto, ) -> MetaResult { - let MetadataManager::V2(mgr) = &self.metadata_manager else { - unreachable!("MetadataManager should be V2") - }; + let mgr = self.metadata_manager.as_v2_ref(); let ctx = StreamContext::from_protobuf(fragment_graph.get_ctx().unwrap()); mgr.catalog_controller @@ -88,7 +88,7 @@ impl DdlController { tracing::error!(id = job_id, error = ?err.as_report(), "failed to create streaming job"); let aborted = mgr .catalog_controller - .try_abort_creating_streaming_job(job_id as _) + .try_abort_creating_streaming_job(job_id as _, false) .await?; if aborted { tracing::warn!(id = job_id, "aborted streaming job"); @@ -194,6 +194,86 @@ impl DdlController { } } + pub async fn drop_object( + &self, + object_type: ObjectType, + object_id: ObjectId, + drop_mode: DropMode, + ) -> MetaResult { + let mgr = self.metadata_manager.as_v2_ref(); + let (release_ctx, version) = match object_type { + ObjectType::Database => mgr.catalog_controller.drop_database(object_id).await?, + ObjectType::Schema => { + return mgr + .catalog_controller + .drop_schema(object_id, drop_mode) + .await; + } + ObjectType::Function => { + return mgr.catalog_controller.drop_function(object_id).await; + } + ObjectType::Connection => { + let (version, conn) = mgr.catalog_controller.drop_connection(object_id).await?; + self.delete_vpc_endpoint(&conn).await?; + return Ok(version); + } + _ => { + mgr.catalog_controller + .drop_relation(object_type, object_id, drop_mode) + .await? + } + }; + + let ReleaseContext { + state_table_ids, + source_ids, + connections, + source_fragments, + removed_actors, + } = release_ctx; + + // delete vpc endpoints. + for conn in connections { + let _ = self + .delete_vpc_endpoint_v2(conn.into_inner()) + .await + .inspect_err(|err| { + tracing::warn!(err = ?err.as_report(), "failed to delete vpc endpoint"); + }); + } + + // unregister sources. + self.source_manager + .unregister_sources(source_ids.into_iter().map(|id| id as _).collect()) + .await; + + // unregister fragments and actors from source manager. + self.source_manager + .drop_source_fragments_v2( + source_fragments + .into_iter() + .map(|(source_id, fragments)| { + ( + source_id as u32, + fragments.into_iter().map(|id| id as u32).collect(), + ) + }) + .collect(), + removed_actors.iter().map(|id| *id as _).collect(), + ) + .await; + + // drop streaming jobs. + self.stream_manager + .drop_streaming_jobs_v2( + removed_actors.into_iter().map(|id| id as _).collect(), + state_table_ids.into_iter().map(|id| id as _).collect(), + ) + .await; + + Ok(version) + } + /// This is used for `ALTER TABLE ADD/DROP COLUMN`. pub async fn replace_table_v2( &self, @@ -201,9 +281,7 @@ impl DdlController { fragment_graph: StreamFragmentGraphProto, table_col_index_mapping: Option, ) -> MetaResult { - let MetadataManager::V2(mgr) = &self.metadata_manager else { - unreachable!("MetadataManager should be V2") - }; + let mgr = self.metadata_manager.as_v2_ref(); let job_id = streaming_job.id(); let _reschedule_job_lock = self.stream_manager.reschedule_lock.read().await; diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index e0a2fb390f42d..40b3d9ef25a11 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -1552,35 +1552,7 @@ impl ScaleController { &self, reschedules: &HashMap, table_parallelism: &HashMap, - ) -> MetaResult>> { - let mut node_dropped_actors = HashMap::new(); - for table_fragments in self - .fragment_manager - .get_fragment_read_guard() - .await - .table_fragments() - .values() - { - for fragment_id in table_fragments.fragments.keys() { - if let Some(reschedule) = reschedules.get(fragment_id) { - for actor_id in &reschedule.removed_actors { - let node_id = table_fragments - .actor_status - .get(actor_id) - .unwrap() - .parallel_unit - .as_ref() - .unwrap() - .worker_node_id; - node_dropped_actors - .entry(node_id as WorkerId) - .or_insert(vec![]) - .push(*actor_id as ActorId); - } - } - } - } - + ) -> MetaResult<()> { // Update fragment info after rescheduling in meta store. self.fragment_manager .post_apply_reschedules(reschedules.clone(), table_parallelism.clone()) @@ -1649,7 +1621,7 @@ impl ScaleController { .await; } - Ok(node_dropped_actors) + Ok(()) } pub async fn generate_table_resize_plan( diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index f3ce3816522b9..8af470ce7df65 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -361,7 +361,7 @@ impl SourceManagerCore { fn drop_source_fragments( &mut self, source_fragments: HashMap>, - actor_splits: &HashSet, + removed_actors: &HashSet, ) { for (source_id, fragment_ids) in source_fragments { if let Entry::Occupied(mut entry) = self.source_fragments.entry(source_id) { @@ -380,7 +380,7 @@ impl SourceManagerCore { } } - for actor_id in actor_splits { + for actor_id in removed_actors { self.actor_splits.remove(actor_id); } } @@ -613,6 +613,15 @@ impl SourceManager { }) } + pub async fn drop_source_fragments_v2( + &self, + source_fragments: HashMap>, + removed_actors: HashSet, + ) { + let mut core = self.core.lock().await; + core.drop_source_fragments(source_fragments, &removed_actors); + } + /// For dropping MV. pub async fn drop_source_fragments(&self, table_fragments: &[TableFragments]) { let mut core = self.core.lock().await; diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 1950a27013f3f..098325f4ad4d6 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; +use std::collections::{BTreeSet, HashMap, HashSet}; use std::sync::Arc; use futures::future::{join_all, try_join_all, BoxFuture}; @@ -36,7 +36,7 @@ use uuid::Uuid; use super::{Locations, RescheduleOptions, ScaleController, ScaleControllerRef, TableResizePolicy}; use crate::barrier::{BarrierScheduler, Command, ReplaceTablePlan}; use crate::hummock::HummockManagerRef; -use crate::manager::{DdlType, MetaSrvEnv, MetadataManager, StreamingJob, WorkerId}; +use crate::manager::{DdlType, MetaSrvEnv, MetadataManager, StreamingJob}; use crate::model::{ActorId, TableFragments, TableParallelism}; use crate::stream::SourceManagerRef; use crate::{MetaError, MetaResult}; @@ -131,18 +131,18 @@ impl CreatingStreamingJobInfo { let mut receivers = HashMap::new(); let mut recovered_job_ids = vec![]; for job_id in job_ids { - if let Some(job) = jobs.get_mut(&job_id) - && let Some(shutdown_tx) = job.shutdown_tx.take() - { - let (tx, rx) = oneshot::channel(); - if shutdown_tx - .send(CreatingState::Canceling { finish_tx: tx }) - .await - .is_ok() - { - receivers.insert(job_id, rx); - } else { - tracing::warn!(id=?job_id, "failed to send canceling state"); + if let Some(job) = jobs.get_mut(&job_id) { + if let Some(shutdown_tx) = job.shutdown_tx.take() { + let (tx, rx) = oneshot::channel(); + if shutdown_tx + .send(CreatingState::Canceling { finish_tx: tx }) + .await + .is_ok() + { + receivers.insert(job_id, rx); + } else { + tracing::warn!(id=?job_id, "failed to send canceling state"); + } } } else { // If these job ids do not exist in streaming_jobs, @@ -331,15 +331,18 @@ impl GlobalStreamManager { }); try_join_all(futures).await?; - self.metadata_manager - .drop_streaming_job_by_ids(&HashSet::from_iter( - std::iter::once(table_id), - )) - .await?; + if let MetadataManager::V1(mgr) = &self.metadata_manager { + mgr.fragment_manager + .drop_table_fragments_vec(&HashSet::from_iter( + std::iter::once(table_id), + )) + .await?; + } } else if !table_fragments.is_created() { tracing::debug!( "cancelling streaming job {table_id} by issue cancel command." ); + self.barrier_scheduler .run_command(Command::CancelStreamingJob(table_fragments)) .await?; @@ -538,14 +541,16 @@ impl GlobalStreamManager { }; if let Err(err) = self.barrier_scheduler.run_command(command).await { - if create_type == CreateType::Foreground { + if create_type == CreateType::Foreground || err.is_cancelled() { let mut table_ids = HashSet::from_iter(std::iter::once(table_id)); if let Some(dummy_table_id) = replace_table_id { table_ids.insert(dummy_table_id); } - self.metadata_manager - .drop_streaming_job_by_ids(&table_ids) - .await?; + if let MetadataManager::V1(mgr) = &self.metadata_manager { + mgr.fragment_manager + .drop_table_fragments_vec(&table_ids) + .await?; + } } return Err(err); @@ -583,9 +588,10 @@ impl GlobalStreamManager { init_split_assignment, })) .await + && let MetadataManager::V1(mgr) = &self.metadata_manager { - self.metadata_manager - .drop_streaming_job_by_ids(&HashSet::from_iter(std::iter::once(dummy_table_id))) + mgr.fragment_manager + .drop_table_fragments_vec(&HashSet::from_iter(std::iter::once(dummy_table_id))) .await?; return Err(err); } @@ -602,15 +608,33 @@ impl GlobalStreamManager { .drop_streaming_jobs_impl(streaming_job_ids) .await .inspect_err(|err| { - tracing::error!(error = %err.as_report(), "Failed to drop streaming jobs"); + tracing::error!(error = ?err.as_report(), "Failed to drop streaming jobs"); }); } } + pub async fn drop_streaming_jobs_v2( + &self, + removed_actors: Vec, + state_table_ids: Vec, + ) { + if !removed_actors.is_empty() { + let _ = self + .barrier_scheduler + .run_command(Command::DropStreamingJobs(removed_actors)) + .await + .inspect_err(|err| { + tracing::error!(error = ?err.as_report(), "failed to run drop command"); + }); + + self.hummock_manager + .unregister_table_ids_fail_fast(&state_table_ids) + .await; + } + } + pub async fn drop_streaming_jobs_impl(&self, table_ids: Vec) -> MetaResult<()> { - let MetadataManager::V1(mgr) = &self.metadata_manager else { - unimplemented!("call drop_streaming_jobs_impl_v2 instead.") - }; + let mgr = self.metadata_manager.as_v1_ref(); let table_fragments_vec = mgr .fragment_manager .select_table_fragments_by_ids(&table_ids) @@ -626,21 +650,17 @@ impl GlobalStreamManager { .await?; // Issues a drop barrier command. - let mut worker_actors = HashMap::new(); - for table_fragments in &table_fragments_vec { - table_fragments - .worker_actor_ids() - .into_iter() - .for_each(|(worker_id, actor_ids)| { - worker_actors - .entry(worker_id) - .or_insert_with(Vec::new) - .extend(actor_ids); - }); - } - self.barrier_scheduler - .run_command(Command::DropStreamingJobs(worker_actors)) - .await?; + let dropped_actors = table_fragments_vec + .iter() + .flat_map(|tf| tf.actor_ids().into_iter()) + .collect_vec(); + let _ = self + .barrier_scheduler + .run_command(Command::DropStreamingJobs(dropped_actors)) + .await + .inspect_err(|err| { + tracing::error!(error = ?err.as_report(), "failed to run drop command"); + }); // Unregister from compaction group afterwards. self.hummock_manager @@ -650,24 +670,6 @@ impl GlobalStreamManager { Ok(()) } - pub fn drop_streaming_jobs_impl_v2( - &self, - _job_info: HashMap>>, - _state_table_ids: Vec, - ) -> MetaResult<()> { - // self.barrier_scheduler.run_command(Command::DropStreamingJobsV2(job_info)).await?; - // - // // TODO: need some refactoring on source manager. - // - // // Unregister from compaction group afterwards. - // self.hummock_manager - // .unregister_table_ids_fail_fast( - // &state_table_ids - // ) - // .await; - unimplemented!("drop_streaming_jobs_impl_v2") - } - /// Cancel streaming jobs and return the canceled table ids. /// 1. Send cancel message to stream jobs (via `cancel_jobs`). /// 2. Send cancel message to recovered stream jobs (via `barrier_scheduler`). @@ -707,10 +709,9 @@ impl GlobalStreamManager { id )))?; } - let MetadataManager::V1(mgr) = &self.metadata_manager else { - unimplemented!("support cancel streaming job in v2"); - }; - mgr.catalog_manager.cancel_create_table_procedure(id.into(), fragment.internal_table_ids()).await?; + if let MetadataManager::V1(mgr) = &self.metadata_manager { + mgr.catalog_manager.cancel_create_table_procedure(id.into(), fragment.internal_table_ids()).await?; + } self.barrier_scheduler .run_command(Command::CancelStreamingJob(fragment)) @@ -721,8 +722,8 @@ impl GlobalStreamManager { tracing::info!(?id, "cancelled recovered streaming job"); Some(id) } - Err(_) => { - tracing::error!(?id, "failed to cancel recovered streaming job, does it correspond to any jobs in `SHOW JOBS`?"); + Err(err) => { + tracing::error!(error=?err.as_report(), "failed to cancel recovered streaming job {id}, does it correspond to any jobs in `SHOW JOBS`?"); None } } From 8919044c0baf3c588796ba33cdc48c2333f3cc40 Mon Sep 17 00:00:00 2001 From: Dylan Date: Tue, 30 Jan 2024 00:25:47 +0800 Subject: [PATCH 05/14] fix(optimizer): fix temporal join shuffle (#14848) --- .../tests/testdata/output/temporal_join.yaml | 13 +++++++------ .../src/optimizer/plan_node/logical_join.rs | 14 +++++++------- 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml index f49a82be2dd78..ea844cda185b1 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml @@ -107,12 +107,13 @@ StreamMaterialize { columns: [k, x1, x2, a1, b1, stream._row_id(hidden), version2.k(hidden)], stream_key: [stream._row_id, k], pk_columns: [stream._row_id, k], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stream.k, stream._row_id) } └─StreamTemporalJoin { type: Inner, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version2.k] } - ├─StreamTemporalJoin { type: Inner, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] } - │ ├─StreamExchange { dist: HashShard(stream.k) } - │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } - │ │ └─StreamTableScan { table: stream, columns: [stream.k, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.k) } - │ └─StreamTableScan { table: version1, columns: [version1.k, version1.x1], pk: [version1.k], dist: UpstreamHashShard(version1.k) } + ├─StreamExchange { dist: HashShard(stream.k) } + │ └─StreamTemporalJoin { type: Inner, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] } + │ ├─StreamExchange { dist: HashShard(stream.k) } + │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } + │ │ └─StreamTableScan { table: stream, columns: [stream.k, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.k) } + │ └─StreamTableScan { table: version1, columns: [version1.k, version1.x1], pk: [version1.k], dist: UpstreamHashShard(version1.k) } └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.k) } └─StreamTableScan { table: version2, columns: [version2.k, version2.x2], pk: [version2.k], dist: UpstreamHashShard(version2.k) } - name: multi-way temporal join with different keys diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index dd555e5e3a1c0..0b83e2491ce99 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -1054,9 +1054,8 @@ impl LogicalJoin { let lookup_prefix_len = reorder_idx.len(); let predicate = predicate.reorder(&reorder_idx); - let left = if dist_key_in_order_key_pos.is_empty() { - self.left() - .to_stream_with_dist_required(&RequiredDist::single(), ctx)? + let required_dist = if dist_key_in_order_key_pos.is_empty() { + RequiredDist::single() } else { let left_eq_indexes = predicate.left_eq_indexes(); let left_dist_key = dist_key_in_order_key_pos @@ -1064,12 +1063,13 @@ impl LogicalJoin { .map(|pos| left_eq_indexes[*pos]) .collect_vec(); - self.left().to_stream_with_dist_required( - &RequiredDist::shard_by_key(self.left().schema().len(), &left_dist_key), - ctx, - )? + RequiredDist::hash_shard(&left_dist_key) }; + let left = self.left().to_stream(ctx)?; + // Enforce a shuffle for the temporal join LHS to let the scheduler be able to schedule the join fragment together with the RHS with a `no_shuffle` exchange. + let left = required_dist.enforce(left, &Order::any()); + if !left.append_only() { return Err(RwError::from(ErrorCode::NotSupported( "Temporal join requires an append-only left input".into(), From 43b6eefcd15803a9b02bac24d3c3758266d0e3eb Mon Sep 17 00:00:00 2001 From: Zihao Xu Date: Mon, 29 Jan 2024 15:38:26 -0500 Subject: [PATCH 06/14] feat(case-when): constant lookup optimization for constant form case-when expression (#14586) --- .../basic/case_when_optimization.slt.part | 242 ++++++++++++++++++ proto/expr.proto | 3 + src/expr/impl/src/scalar/case.rs | 132 +++++++++- .../input/case_when_optimization.yaml | 117 +++++++++ .../output/case_when_optimization.yaml | 123 +++++++++ src/frontend/src/binder/expr/mod.rs | 76 ++++++ src/frontend/src/expr/pure.rs | 1 + src/frontend/src/expr/type_inference/func.rs | 15 ++ 8 files changed, 707 insertions(+), 2 deletions(-) create mode 100644 e2e_test/batch/basic/case_when_optimization.slt.part create mode 100644 src/frontend/planner_test/tests/testdata/input/case_when_optimization.yaml create mode 100644 src/frontend/planner_test/tests/testdata/output/case_when_optimization.yaml diff --git a/e2e_test/batch/basic/case_when_optimization.slt.part b/e2e_test/batch/basic/case_when_optimization.slt.part new file mode 100644 index 0000000000000..7e01be030a911 --- /dev/null +++ b/e2e_test/batch/basic/case_when_optimization.slt.part @@ -0,0 +1,242 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +CREATE TABLE t1 (c1 INT, c2 INT, c3 INT); + +statement ok +INSERT INTO t1 VALUES (1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (5, 5, 5), + (6, 6, 6), (7, 7, 7), (8, 8, 8), (9, 9, 9), (10, 10, 10), + (11, 11, 11), (12, 12, 12), (13, 13, 13), (14, 14, 14), (15, 15, 15), + (16, 16, 16), (17, 17, 17), (18, 18, 18), (19, 19, 19), (20, 20, 20), + (21, 21, 21), (22, 22, 22), (23, 23, 23), (24, 24, 24), (25, 25, 25), + (26, 26, 26), (27, 27, 27), (28, 28, 28), (29, 29, 29), (30, 30, 30), + (31, 31, 31), (32, 32, 32), (33, 33, 33), (34, 34, 34), (35, 35, 35), + (36, 36, 36), (37, 37, 37), (38, 38, 38), (39, 39, 39), (40, 40, 40), + (41, 41, 41), (42, 42, 42), (43, 43, 43), (44, 44, 44), (45, 45, 45), + (46, 46, 46), (47, 47, 47), (48, 48, 48), (49, 49, 49), (50, 50, 50), + (51, 51, 51), (52, 52, 52), (53, 53, 53), (54, 54, 54), (55, 55, 55), + (56, 56, 56), (57, 57, 57), (58, 58, 58), (59, 59, 59), (60, 60, 60), + (61, 61, 61), (62, 62, 62), (63, 63, 63), (64, 64, 64), (65, 65, 65), + (66, 66, 66), (67, 67, 67), (68, 68, 68), (69, 69, 69), (70, 70, 70), + (71, 71, 71), (72, 72, 72), (73, 73, 73), (74, 74, 74), (75, 75, 75), + (76, 76, 76), (77, 77, 77), (78, 78, 78), (79, 79, 79), (80, 80, 80), + (81, 81, 81), (82, 82, 82), (83, 83, 83), (84, 84, 84), (85, 85, 85), + (86, 86, 86), (87, 87, 87), (88, 88, 88), (89, 89, 89), (90, 90, 90), + (91, 91, 91), (92, 92, 92), (93, 93, 93), (94, 94, 94), (95, 95, 95), + (96, 96, 96), (97, 97, 97), (98, 98, 98), (99, 99, 99), (100, 100, 100); + + +# 101 arms case-when expression, with optimizable pattern +query I +SELECT + CASE c1 + WHEN 1 THEN 'one' + WHEN 2 THEN 'two' + WHEN 3 THEN 'three' + WHEN 4 THEN 'four' + WHEN 5 THEN 'five' + WHEN 6 THEN 'six' + WHEN 7 THEN 'seven' + WHEN 8 THEN 'eight' + WHEN 9 THEN 'nine' + WHEN 10 THEN 'ten' + WHEN 11 THEN 'eleven' + WHEN 12 THEN 'twelve' + WHEN 13 THEN 'thirteen' + WHEN 14 THEN 'fourteen' + WHEN 15 THEN 'fifteen' + WHEN 16 THEN 'sixteen' + WHEN 17 THEN 'seventeen' + WHEN 18 THEN 'eighteen' + WHEN 19 THEN 'nineteen' + WHEN 20 THEN 'twenty' + WHEN 21 THEN 'twenty-one' + WHEN 22 THEN 'twenty-two' + WHEN 23 THEN 'twenty-three' + WHEN 24 THEN 'twenty-four' + WHEN 25 THEN 'twenty-five' + WHEN 26 THEN 'twenty-six' + WHEN 27 THEN 'twenty-seven' + WHEN 28 THEN 'twenty-eight' + WHEN 29 THEN 'twenty-nine' + WHEN 30 THEN 'thirty' + WHEN 31 THEN 'thirty-one' + WHEN 32 THEN 'thirty-two' + WHEN 33 THEN 'thirty-three' + WHEN 34 THEN 'thirty-four' + WHEN 35 THEN 'thirty-five' + WHEN 36 THEN 'thirty-six' + WHEN 37 THEN 'thirty-seven' + WHEN 38 THEN 'thirty-eight' + WHEN 39 THEN 'thirty-nine' + WHEN 40 THEN 'forty' + WHEN 41 THEN 'forty-one' + WHEN 42 THEN 'forty-two' + WHEN 43 THEN 'forty-three' + WHEN 44 THEN 'forty-four' + WHEN 45 THEN 'forty-five' + WHEN 46 THEN 'forty-six' + WHEN 47 THEN 'forty-seven' + WHEN 48 THEN 'forty-eight' + WHEN 49 THEN 'forty-nine' + WHEN 50 THEN 'fifty' + WHEN 51 THEN 'fifty-one' + WHEN 52 THEN 'fifty-two' + WHEN 53 THEN 'fifty-three' + WHEN 54 THEN 'fifty-four' + WHEN 55 THEN 'fifty-five' + WHEN 56 THEN 'fifty-six' + WHEN 57 THEN 'fifty-seven' + WHEN 58 THEN 'fifty-eight' + WHEN 59 THEN 'fifty-nine' + WHEN 60 THEN 'sixty' + WHEN 61 THEN 'sixty-one' + WHEN 62 THEN 'sixty-two' + WHEN 63 THEN 'sixty-three' + WHEN 64 THEN 'sixty-four' + WHEN 65 THEN 'sixty-five' + WHEN 66 THEN 'sixty-six' + WHEN 67 THEN 'sixty-seven' + WHEN 68 THEN 'sixty-eight' + WHEN 69 THEN 'sixty-nine' + WHEN 70 THEN 'seventy' + WHEN 71 THEN 'seventy-one' + WHEN 72 THEN 'seventy-two' + WHEN 73 THEN 'seventy-three' + WHEN 74 THEN 'seventy-four' + WHEN 75 THEN 'seventy-five' + WHEN 76 THEN 'seventy-six' + WHEN 77 THEN 'seventy-seven' + WHEN 78 THEN 'seventy-eight' + WHEN 79 THEN 'seventy-nine' + WHEN 80 THEN 'eighty' + WHEN 81 THEN 'eighty-one' + WHEN 82 THEN 'eighty-two' + WHEN 83 THEN 'eighty-three' + WHEN 84 THEN 'eighty-four' + WHEN 85 THEN 'eighty-five' + WHEN 86 THEN 'eighty-six' + WHEN 87 THEN 'eighty-seven' + WHEN 88 THEN 'eighty-eight' + WHEN 89 THEN 'eighty-nine' + WHEN 90 THEN 'ninety' + WHEN 91 THEN 'ninety-one' + WHEN 92 THEN 'ninety-two' + WHEN 93 THEN 'ninety-three' + WHEN 94 THEN 'ninety-four' + WHEN 95 THEN 'ninety-five' + WHEN 96 THEN 'ninety-six' + WHEN 97 THEN 'ninety-seven' + WHEN 98 THEN 'ninety-eight' + WHEN 99 THEN 'ninety-nine' + WHEN 100 THEN 'one hundred' + ELSE + '114514' + END +FROM t1 +ORDER BY c1 ASC; +---- +one +two +three +four +five +six +seven +eight +nine +ten +eleven +twelve +thirteen +fourteen +fifteen +sixteen +seventeen +eighteen +nineteen +twenty +twenty-one +twenty-two +twenty-three +twenty-four +twenty-five +twenty-six +twenty-seven +twenty-eight +twenty-nine +thirty +thirty-one +thirty-two +thirty-three +thirty-four +thirty-five +thirty-six +thirty-seven +thirty-eight +thirty-nine +forty +forty-one +forty-two +forty-three +forty-four +forty-five +forty-six +forty-seven +forty-eight +forty-nine +fifty +fifty-one +fifty-two +fifty-three +fifty-four +fifty-five +fifty-six +fifty-seven +fifty-eight +fifty-nine +sixty +sixty-one +sixty-two +sixty-three +sixty-four +sixty-five +sixty-six +sixty-seven +sixty-eight +sixty-nine +seventy +seventy-one +seventy-two +seventy-three +seventy-four +seventy-five +seventy-six +seventy-seven +seventy-eight +seventy-nine +eighty +eighty-one +eighty-two +eighty-three +eighty-four +eighty-five +eighty-six +eighty-seven +eighty-eight +eighty-nine +ninety +ninety-one +ninety-two +ninety-three +ninety-four +ninety-five +ninety-six +ninety-seven +ninety-eight +ninety-nine +one hundred + +statement ok +drop table t1; \ No newline at end of file diff --git a/proto/expr.proto b/proto/expr.proto index f62ee2936d115..7ab48a405d139 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -82,6 +82,9 @@ message ExprNode { LTRIM = 210; RTRIM = 211; CASE = 212; + // Optimize case-when expression to constant lookup + // when arms are in a large scale with simple form + CONSTANT_LOOKUP = 624; // ROUND(numeric, integer) -> numeric ROUND_DIGIT = 213; // ROUND(numeric) -> numeric diff --git a/src/expr/impl/src/scalar/case.rs b/src/expr/impl/src/scalar/case.rs index d950cd60af55a..64a68b987f860 100644 --- a/src/expr/impl/src/scalar/case.rs +++ b/src/expr/impl/src/scalar/case.rs @@ -12,12 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::sync::Arc; use risingwave_common::array::{ArrayRef, DataChunk}; use risingwave_common::bail; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, Datum}; +use risingwave_common::row::{OwnedRow, Row}; +use risingwave_common::types::{DataType, Datum, ScalarImpl}; use risingwave_expr::expr::{BoxedExpression, Expression}; use risingwave_expr::{build_function, Result}; @@ -106,6 +107,132 @@ impl Expression for CaseExpression { } } +/// With large scale of simple form match arms in case-when expression, +/// we could optimize the `CaseExpression` to `ConstantLookupExpression`, +/// which could significantly facilitate the evaluation of case-when. +#[derive(Debug)] +struct ConstantLookupExpression { + return_type: DataType, + arms: HashMap, + fallback: Option, + /// `operand` must exist at present + operand: BoxedExpression, +} + +impl ConstantLookupExpression { + fn new( + return_type: DataType, + arms: HashMap, + fallback: Option, + operand: BoxedExpression, + ) -> Self { + Self { + return_type, + arms, + fallback, + operand, + } + } +} + +#[async_trait::async_trait] +impl Expression for ConstantLookupExpression { + fn return_type(&self) -> DataType { + self.return_type.clone() + } + + async fn eval(&self, input: &DataChunk) -> Result { + let input_len = input.capacity(); + let mut builder = self.return_type().create_array_builder(input_len); + + // Evaluate the input DataChunk at first + let eval_result = self.operand.eval(input).await?; + + for i in 0..input_len { + let datum = eval_result.datum_at(i); + let (row, vis) = input.row_at(i); + + // Check for visibility + if !vis { + builder.append_null(); + continue; + } + + // Note that the `owned_row` here is extracted from input + // rather than from `eval_result` + let owned_row = row.into_owned_row(); + + if let Some(expr) = self.arms.get(datum.as_ref().unwrap()) { + builder.append(expr.eval_row(&owned_row).await.unwrap().as_ref()); + } else { + // Otherwise this should goes to the fallback arm + // The fallback arm should also be const + if let Some(ref fallback) = self.fallback { + builder.append(fallback.eval_row(&owned_row).await.unwrap().as_ref()); + } else { + builder.append_null(); + } + } + } + + Ok(Arc::new(builder.finish())) + } + + async fn eval_row(&self, input: &OwnedRow) -> Result { + let datum = self.operand.eval_row(input).await?; + + if let Some(expr) = self.arms.get(datum.as_ref().unwrap()) { + expr.eval_row(input).await + } else { + let Some(ref expr) = self.fallback else { + return Ok(None); + }; + expr.eval_row(input).await + } + } +} + +#[build_function("constant_lookup(...) -> any", type_infer = "panic")] +fn build_constant_lookup_expr( + return_type: DataType, + children: Vec, +) -> Result { + if children.is_empty() { + bail!("children expression must not be empty for constant lookup expression"); + } + + let mut children = children; + + let operand = children.remove(0); + + let mut arms = HashMap::new(); + + // Build the `arms` with iterating over `when` & `then` clauses + let mut iter = children.into_iter().array_chunks(); + for [when, then] in iter.by_ref() { + let Ok(Some(s)) = when.eval_const() else { + bail!("expect when expression to be const"); + }; + arms.insert(s, then); + } + + let fallback = if let Some(else_clause) = iter.into_remainder().unwrap().next() { + if else_clause.return_type() != return_type { + bail!("Type mismatched between else and case."); + } + Some(else_clause) + } else { + None + }; + + Ok(Box::new(ConstantLookupExpression::new( + return_type, + arms, + fallback, + operand, + ))) +} + #[build_function("case(...) -> any", type_infer = "panic")] fn build_case_expr( return_type: DataType, @@ -132,6 +259,7 @@ fn build_case_expr( } else { None }; + Ok(Box::new(CaseExpression::new( return_type, when_clauses, diff --git a/src/frontend/planner_test/tests/testdata/input/case_when_optimization.yaml b/src/frontend/planner_test/tests/testdata/input/case_when_optimization.yaml new file mode 100644 index 0000000000000..d37f3395e02d4 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/case_when_optimization.yaml @@ -0,0 +1,117 @@ +- id: create_table + sql: | + CREATE TABLE t1 (c1 INT, c2 INT, c3 INT); + expected_outputs: [] +- id: basic_optimization_pattern + before: + - create_table + sql: | + SELECT + CASE c1 + WHEN 1 THEN 'one' + WHEN 2 THEN 'two' + WHEN 3 THEN 'three' + WHEN 4 THEN 'four' + WHEN 5 THEN 'five' + WHEN 6 THEN 'six' + WHEN 7 THEN 'seven' + WHEN 8 THEN 'eight' + WHEN 9 THEN 'nine' + WHEN 10 THEN 'ten' + WHEN 11 THEN 'eleven' + WHEN 12 THEN 'twelve' + WHEN 13 THEN 'thirteen' + WHEN 14 THEN 'fourteen' + WHEN 15 THEN 'fifteen' + WHEN 16 THEN 'sixteen' + WHEN 17 THEN 'seventeen' + WHEN 18 THEN 'eighteen' + WHEN 19 THEN 'nineteen' + WHEN 20 THEN 'twenty' + WHEN 21 THEN 'twenty-one' + WHEN 22 THEN 'twenty-two' + WHEN 23 THEN 'twenty-three' + WHEN 24 THEN 'twenty-four' + WHEN 25 THEN 'twenty-five' + WHEN 26 THEN 'twenty-six' + WHEN 27 THEN 'twenty-seven' + WHEN 28 THEN 'twenty-eight' + WHEN 29 THEN 'twenty-nine' + WHEN 30 THEN 'thirty' + WHEN 31 THEN 'thirty-one' + WHEN 32 THEN 'thirty-two' + WHEN 33 THEN 'thirty-three' + WHEN 34 THEN 'thirty-four' + WHEN 35 THEN 'thirty-five' + WHEN 36 THEN 'thirty-six' + WHEN 37 THEN 'thirty-seven' + WHEN 38 THEN 'thirty-eight' + WHEN 39 THEN 'thirty-nine' + WHEN 40 THEN 'forty' + WHEN 41 THEN 'forty-one' + WHEN 42 THEN 'forty-two' + WHEN 43 THEN 'forty-three' + WHEN 44 THEN 'forty-four' + WHEN 45 THEN 'forty-five' + WHEN 46 THEN 'forty-six' + WHEN 47 THEN 'forty-seven' + WHEN 48 THEN 'forty-eight' + WHEN 49 THEN 'forty-nine' + WHEN 50 THEN 'fifty' + WHEN 51 THEN 'fifty-one' + WHEN 52 THEN 'fifty-two' + WHEN 53 THEN 'fifty-three' + WHEN 54 THEN 'fifty-four' + WHEN 55 THEN 'fifty-five' + WHEN 56 THEN 'fifty-six' + WHEN 57 THEN 'fifty-seven' + WHEN 58 THEN 'fifty-eight' + WHEN 59 THEN 'fifty-nine' + WHEN 60 THEN 'sixty' + WHEN 61 THEN 'sixty-one' + WHEN 62 THEN 'sixty-two' + WHEN 63 THEN 'sixty-three' + WHEN 64 THEN 'sixty-four' + WHEN 65 THEN 'sixty-five' + WHEN 66 THEN 'sixty-six' + WHEN 67 THEN 'sixty-seven' + WHEN 68 THEN 'sixty-eight' + WHEN 69 THEN 'sixty-nine' + WHEN 70 THEN 'seventy' + WHEN 71 THEN 'seventy-one' + WHEN 72 THEN 'seventy-two' + WHEN 73 THEN 'seventy-three' + WHEN 74 THEN 'seventy-four' + WHEN 75 THEN 'seventy-five' + WHEN 76 THEN 'seventy-six' + WHEN 77 THEN 'seventy-seven' + WHEN 78 THEN 'seventy-eight' + WHEN 79 THEN 'seventy-nine' + WHEN 80 THEN 'eighty' + WHEN 81 THEN 'eighty-one' + WHEN 82 THEN 'eighty-two' + WHEN 83 THEN 'eighty-three' + WHEN 84 THEN 'eighty-four' + WHEN 85 THEN 'eighty-five' + WHEN 86 THEN 'eighty-six' + WHEN 87 THEN 'eighty-seven' + WHEN 88 THEN 'eighty-eight' + WHEN 89 THEN 'eighty-nine' + WHEN 90 THEN 'ninety' + WHEN 91 THEN 'ninety-one' + WHEN 92 THEN 'ninety-two' + WHEN 93 THEN 'ninety-three' + WHEN 94 THEN 'ninety-four' + WHEN 95 THEN 'ninety-five' + WHEN 96 THEN 'ninety-six' + WHEN 97 THEN 'ninety-seven' + WHEN 98 THEN 'ninety-eight' + WHEN 99 THEN 'ninety-nine' + WHEN 100 THEN 'one hundred' + ELSE + '114514' + END + FROM t1; + expected_outputs: + - logical_plan + - batch_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/case_when_optimization.yaml b/src/frontend/planner_test/tests/testdata/output/case_when_optimization.yaml new file mode 100644 index 0000000000000..82f0ee33aaeb4 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/case_when_optimization.yaml @@ -0,0 +1,123 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. +- id: create_table + sql: | + CREATE TABLE t1 (c1 INT, c2 INT, c3 INT); +- id: basic_optimization_pattern + before: + - create_table + sql: | + SELECT + CASE c1 + WHEN 1 THEN 'one' + WHEN 2 THEN 'two' + WHEN 3 THEN 'three' + WHEN 4 THEN 'four' + WHEN 5 THEN 'five' + WHEN 6 THEN 'six' + WHEN 7 THEN 'seven' + WHEN 8 THEN 'eight' + WHEN 9 THEN 'nine' + WHEN 10 THEN 'ten' + WHEN 11 THEN 'eleven' + WHEN 12 THEN 'twelve' + WHEN 13 THEN 'thirteen' + WHEN 14 THEN 'fourteen' + WHEN 15 THEN 'fifteen' + WHEN 16 THEN 'sixteen' + WHEN 17 THEN 'seventeen' + WHEN 18 THEN 'eighteen' + WHEN 19 THEN 'nineteen' + WHEN 20 THEN 'twenty' + WHEN 21 THEN 'twenty-one' + WHEN 22 THEN 'twenty-two' + WHEN 23 THEN 'twenty-three' + WHEN 24 THEN 'twenty-four' + WHEN 25 THEN 'twenty-five' + WHEN 26 THEN 'twenty-six' + WHEN 27 THEN 'twenty-seven' + WHEN 28 THEN 'twenty-eight' + WHEN 29 THEN 'twenty-nine' + WHEN 30 THEN 'thirty' + WHEN 31 THEN 'thirty-one' + WHEN 32 THEN 'thirty-two' + WHEN 33 THEN 'thirty-three' + WHEN 34 THEN 'thirty-four' + WHEN 35 THEN 'thirty-five' + WHEN 36 THEN 'thirty-six' + WHEN 37 THEN 'thirty-seven' + WHEN 38 THEN 'thirty-eight' + WHEN 39 THEN 'thirty-nine' + WHEN 40 THEN 'forty' + WHEN 41 THEN 'forty-one' + WHEN 42 THEN 'forty-two' + WHEN 43 THEN 'forty-three' + WHEN 44 THEN 'forty-four' + WHEN 45 THEN 'forty-five' + WHEN 46 THEN 'forty-six' + WHEN 47 THEN 'forty-seven' + WHEN 48 THEN 'forty-eight' + WHEN 49 THEN 'forty-nine' + WHEN 50 THEN 'fifty' + WHEN 51 THEN 'fifty-one' + WHEN 52 THEN 'fifty-two' + WHEN 53 THEN 'fifty-three' + WHEN 54 THEN 'fifty-four' + WHEN 55 THEN 'fifty-five' + WHEN 56 THEN 'fifty-six' + WHEN 57 THEN 'fifty-seven' + WHEN 58 THEN 'fifty-eight' + WHEN 59 THEN 'fifty-nine' + WHEN 60 THEN 'sixty' + WHEN 61 THEN 'sixty-one' + WHEN 62 THEN 'sixty-two' + WHEN 63 THEN 'sixty-three' + WHEN 64 THEN 'sixty-four' + WHEN 65 THEN 'sixty-five' + WHEN 66 THEN 'sixty-six' + WHEN 67 THEN 'sixty-seven' + WHEN 68 THEN 'sixty-eight' + WHEN 69 THEN 'sixty-nine' + WHEN 70 THEN 'seventy' + WHEN 71 THEN 'seventy-one' + WHEN 72 THEN 'seventy-two' + WHEN 73 THEN 'seventy-three' + WHEN 74 THEN 'seventy-four' + WHEN 75 THEN 'seventy-five' + WHEN 76 THEN 'seventy-six' + WHEN 77 THEN 'seventy-seven' + WHEN 78 THEN 'seventy-eight' + WHEN 79 THEN 'seventy-nine' + WHEN 80 THEN 'eighty' + WHEN 81 THEN 'eighty-one' + WHEN 82 THEN 'eighty-two' + WHEN 83 THEN 'eighty-three' + WHEN 84 THEN 'eighty-four' + WHEN 85 THEN 'eighty-five' + WHEN 86 THEN 'eighty-six' + WHEN 87 THEN 'eighty-seven' + WHEN 88 THEN 'eighty-eight' + WHEN 89 THEN 'eighty-nine' + WHEN 90 THEN 'ninety' + WHEN 91 THEN 'ninety-one' + WHEN 92 THEN 'ninety-two' + WHEN 93 THEN 'ninety-three' + WHEN 94 THEN 'ninety-four' + WHEN 95 THEN 'ninety-five' + WHEN 96 THEN 'ninety-six' + WHEN 97 THEN 'ninety-seven' + WHEN 98 THEN 'ninety-eight' + WHEN 99 THEN 'ninety-nine' + WHEN 100 THEN 'one hundred' + ELSE + '114514' + END + FROM t1; + logical_plan: |- + LogicalProject + ├─exprs: ConstantLookup(t1.c1, 1:Int32, 'one':Varchar, 2:Int32, 'two':Varchar, 3:Int32, 'three':Varchar, 4:Int32, 'four':Varchar, 5:Int32, 'five':Varchar, 6:Int32, 'six':Varchar, 7:Int32, 'seven':Varchar, 8:Int32, 'eight':Varchar, 9:Int32, 'nine':Varchar, 10:Int32, 'ten':Varchar, 11:Int32, 'eleven':Varchar, 12:Int32, 'twelve':Varchar, 13:Int32, 'thirteen':Varchar, 14:Int32, 'fourteen':Varchar, 15:Int32, 'fifteen':Varchar, 16:Int32, 'sixteen':Varchar, 17:Int32, 'seventeen':Varchar, 18:Int32, 'eighteen':Varchar, 19:Int32, 'nineteen':Varchar, 20:Int32, 'twenty':Varchar, 21:Int32, 'twenty-one':Varchar, 22:Int32, 'twenty-two':Varchar, 23:Int32, 'twenty-three':Varchar, 24:Int32, 'twenty-four':Varchar, 25:Int32, 'twenty-five':Varchar, 26:Int32, 'twenty-six':Varchar, 27:Int32, 'twenty-seven':Varchar, 28:Int32, 'twenty-eight':Varchar, 29:Int32, 'twenty-nine':Varchar, 30:Int32, 'thirty':Varchar, 31:Int32, 'thirty-one':Varchar, 32:Int32, 'thirty-two':Varchar, 33:Int32, 'thirty-three':Varchar, 34:Int32, 'thirty-four':Varchar, 35:Int32, 'thirty-five':Varchar, 36:Int32, 'thirty-six':Varchar, 37:Int32, 'thirty-seven':Varchar, 38:Int32, 'thirty-eight':Varchar, 39:Int32, 'thirty-nine':Varchar, 40:Int32, 'forty':Varchar, 41:Int32, 'forty-one':Varchar, 42:Int32, 'forty-two':Varchar, 43:Int32, 'forty-three':Varchar, 44:Int32, 'forty-four':Varchar, 45:Int32, 'forty-five':Varchar, 46:Int32, 'forty-six':Varchar, 47:Int32, 'forty-seven':Varchar, 48:Int32, 'forty-eight':Varchar, 49:Int32, 'forty-nine':Varchar, 50:Int32, 'fifty':Varchar, 51:Int32, 'fifty-one':Varchar, 52:Int32, 'fifty-two':Varchar, 53:Int32, 'fifty-three':Varchar, 54:Int32, 'fifty-four':Varchar, 55:Int32, 'fifty-five':Varchar, 56:Int32, 'fifty-six':Varchar, 57:Int32, 'fifty-seven':Varchar, 58:Int32, 'fifty-eight':Varchar, 59:Int32, 'fifty-nine':Varchar, 60:Int32, 'sixty':Varchar, 61:Int32, 'sixty-one':Varchar, 62:Int32, 'sixty-two':Varchar, 63:Int32, 'sixty-three':Varchar, 64:Int32, 'sixty-four':Varchar, 65:Int32, 'sixty-five':Varchar, 66:Int32, 'sixty-six':Varchar, 67:Int32, 'sixty-seven':Varchar, 68:Int32, 'sixty-eight':Varchar, 69:Int32, 'sixty-nine':Varchar, 70:Int32, 'seventy':Varchar, 71:Int32, 'seventy-one':Varchar, 72:Int32, 'seventy-two':Varchar, 73:Int32, 'seventy-three':Varchar, 74:Int32, 'seventy-four':Varchar, 75:Int32, 'seventy-five':Varchar, 76:Int32, 'seventy-six':Varchar, 77:Int32, 'seventy-seven':Varchar, 78:Int32, 'seventy-eight':Varchar, 79:Int32, 'seventy-nine':Varchar, 80:Int32, 'eighty':Varchar, 81:Int32, 'eighty-one':Varchar, 82:Int32, 'eighty-two':Varchar, 83:Int32, 'eighty-three':Varchar, 84:Int32, 'eighty-four':Varchar, 85:Int32, 'eighty-five':Varchar, 86:Int32, 'eighty-six':Varchar, 87:Int32, 'eighty-seven':Varchar, 88:Int32, 'eighty-eight':Varchar, 89:Int32, 'eighty-nine':Varchar, 90:Int32, 'ninety':Varchar, 91:Int32, 'ninety-one':Varchar, 92:Int32, 'ninety-two':Varchar, 93:Int32, 'ninety-three':Varchar, 94:Int32, 'ninety-four':Varchar, 95:Int32, 'ninety-five':Varchar, 96:Int32, 'ninety-six':Varchar, 97:Int32, 'ninety-seven':Varchar, 98:Int32, 'ninety-eight':Varchar, 99:Int32, 'ninety-nine':Varchar, 100:Int32, 'one hundred':Varchar, '114514':Varchar) as $expr1 + └─LogicalScan { table: t1, columns: [t1.c1, t1.c2, t1.c3, t1._row_id] } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject + ├─exprs: ConstantLookup(t1.c1, 1:Int32, 'one':Varchar, 2:Int32, 'two':Varchar, 3:Int32, 'three':Varchar, 4:Int32, 'four':Varchar, 5:Int32, 'five':Varchar, 6:Int32, 'six':Varchar, 7:Int32, 'seven':Varchar, 8:Int32, 'eight':Varchar, 9:Int32, 'nine':Varchar, 10:Int32, 'ten':Varchar, 11:Int32, 'eleven':Varchar, 12:Int32, 'twelve':Varchar, 13:Int32, 'thirteen':Varchar, 14:Int32, 'fourteen':Varchar, 15:Int32, 'fifteen':Varchar, 16:Int32, 'sixteen':Varchar, 17:Int32, 'seventeen':Varchar, 18:Int32, 'eighteen':Varchar, 19:Int32, 'nineteen':Varchar, 20:Int32, 'twenty':Varchar, 21:Int32, 'twenty-one':Varchar, 22:Int32, 'twenty-two':Varchar, 23:Int32, 'twenty-three':Varchar, 24:Int32, 'twenty-four':Varchar, 25:Int32, 'twenty-five':Varchar, 26:Int32, 'twenty-six':Varchar, 27:Int32, 'twenty-seven':Varchar, 28:Int32, 'twenty-eight':Varchar, 29:Int32, 'twenty-nine':Varchar, 30:Int32, 'thirty':Varchar, 31:Int32, 'thirty-one':Varchar, 32:Int32, 'thirty-two':Varchar, 33:Int32, 'thirty-three':Varchar, 34:Int32, 'thirty-four':Varchar, 35:Int32, 'thirty-five':Varchar, 36:Int32, 'thirty-six':Varchar, 37:Int32, 'thirty-seven':Varchar, 38:Int32, 'thirty-eight':Varchar, 39:Int32, 'thirty-nine':Varchar, 40:Int32, 'forty':Varchar, 41:Int32, 'forty-one':Varchar, 42:Int32, 'forty-two':Varchar, 43:Int32, 'forty-three':Varchar, 44:Int32, 'forty-four':Varchar, 45:Int32, 'forty-five':Varchar, 46:Int32, 'forty-six':Varchar, 47:Int32, 'forty-seven':Varchar, 48:Int32, 'forty-eight':Varchar, 49:Int32, 'forty-nine':Varchar, 50:Int32, 'fifty':Varchar, 51:Int32, 'fifty-one':Varchar, 52:Int32, 'fifty-two':Varchar, 53:Int32, 'fifty-three':Varchar, 54:Int32, 'fifty-four':Varchar, 55:Int32, 'fifty-five':Varchar, 56:Int32, 'fifty-six':Varchar, 57:Int32, 'fifty-seven':Varchar, 58:Int32, 'fifty-eight':Varchar, 59:Int32, 'fifty-nine':Varchar, 60:Int32, 'sixty':Varchar, 61:Int32, 'sixty-one':Varchar, 62:Int32, 'sixty-two':Varchar, 63:Int32, 'sixty-three':Varchar, 64:Int32, 'sixty-four':Varchar, 65:Int32, 'sixty-five':Varchar, 66:Int32, 'sixty-six':Varchar, 67:Int32, 'sixty-seven':Varchar, 68:Int32, 'sixty-eight':Varchar, 69:Int32, 'sixty-nine':Varchar, 70:Int32, 'seventy':Varchar, 71:Int32, 'seventy-one':Varchar, 72:Int32, 'seventy-two':Varchar, 73:Int32, 'seventy-three':Varchar, 74:Int32, 'seventy-four':Varchar, 75:Int32, 'seventy-five':Varchar, 76:Int32, 'seventy-six':Varchar, 77:Int32, 'seventy-seven':Varchar, 78:Int32, 'seventy-eight':Varchar, 79:Int32, 'seventy-nine':Varchar, 80:Int32, 'eighty':Varchar, 81:Int32, 'eighty-one':Varchar, 82:Int32, 'eighty-two':Varchar, 83:Int32, 'eighty-three':Varchar, 84:Int32, 'eighty-four':Varchar, 85:Int32, 'eighty-five':Varchar, 86:Int32, 'eighty-six':Varchar, 87:Int32, 'eighty-seven':Varchar, 88:Int32, 'eighty-eight':Varchar, 89:Int32, 'eighty-nine':Varchar, 90:Int32, 'ninety':Varchar, 91:Int32, 'ninety-one':Varchar, 92:Int32, 'ninety-two':Varchar, 93:Int32, 'ninety-three':Varchar, 94:Int32, 'ninety-four':Varchar, 95:Int32, 'ninety-five':Varchar, 96:Int32, 'ninety-six':Varchar, 97:Int32, 'ninety-seven':Varchar, 98:Int32, 'ninety-eight':Varchar, 99:Int32, 'ninety-nine':Varchar, 100:Int32, 'one hundred':Varchar, '114514':Varchar) as $expr1 + └─BatchScan { table: t1, columns: [t1.c1], distribution: SomeShard } diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 93356cd1efec6..7da1518ceb552 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -35,6 +35,13 @@ mod order_by; mod subquery; mod value; +/// The limit arms for case-when expression +/// When the number of condition arms exceed +/// this limit, we will try optimize the case-when +/// expression to `ConstantLookupExpression` +/// Check `case.rs` for details. +const CASE_WHEN_ARMS_OPTIMIZE_LIMIT: usize = 30; + impl Binder { /// Bind an expression with `bind_expr_inner`, attach the original expression /// to the error message. @@ -462,6 +469,56 @@ impl Binder { Ok(func_call.into()) } + /// The helper function to check if the current case-when + /// expression in `bind_case` could be optimized + /// into `ConstantLookupExpression` + fn check_bind_case_optimization( + &mut self, + conditions: Vec, + results_expr: Vec, + operand: Option>, + fallback: Option, + constant_lookup_inputs: &mut Vec, + ) -> bool { + if conditions.len() < CASE_WHEN_ARMS_OPTIMIZE_LIMIT { + return false; + } + + // TODO(Zihao): we could possibly optimize some simple cases when + // `operand` is None in the future, the current choice is not conducting the optimization. + // e.g., select case when c1 = 1 then (...) when (same pattern) then (... ) [else (...)] end from t1; + if let Some(operand) = operand { + let Ok(operand) = self.bind_expr_inner(*operand) else { + return false; + }; + constant_lookup_inputs.push(operand); + } else { + return false; + } + + for (condition, result) in zip_eq_fast(conditions, results_expr) { + if let Expr::Value(_) = condition.clone() { + let Ok(input) = self.bind_expr_inner(condition.clone()) else { + return false; + }; + constant_lookup_inputs.push(input); + } else { + // If at least one condition is not in the simple form / not constant, + // we can NOT do the subsequent optimization then + return false; + } + + constant_lookup_inputs.push(result); + } + + // The fallback arm for case-when expression + if let Some(expr) = fallback { + constant_lookup_inputs.push(expr); + } + + true + } + pub(super) fn bind_case( &mut self, operand: Option>, @@ -478,6 +535,21 @@ impl Binder { .map(|expr| self.bind_expr_inner(*expr)) .transpose()?; + let mut constant_lookup_inputs = Vec::new(); + + // See if the case-when expression can be optimized + let optimize_flag = self.check_bind_case_optimization( + conditions.clone(), + results_expr.clone(), + operand.clone(), + else_result_expr.clone(), + &mut constant_lookup_inputs, + ); + + if optimize_flag { + return Ok(FunctionCall::new(ExprType::ConstantLookup, constant_lookup_inputs)?.into()); + } + for (condition, result) in zip_eq_fast(conditions, results_expr) { let condition = match operand { Some(ref t) => Expr::BinaryOp { @@ -493,14 +565,18 @@ impl Binder { ); inputs.push(result); } + + // The fallback arm for case-when expression if let Some(expr) = else_result_expr { inputs.push(expr); } + if inputs.iter().any(ExprImpl::has_table_function) { return Err( ErrorCode::BindError("table functions are not allowed in CASE".into()).into(), ); } + Ok(FunctionCall::new(ExprType::Case, inputs)?.into()) } diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index 7e7378a65752f..b87c6d18feb10 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -82,6 +82,7 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::Ltrim | expr_node::Type::Rtrim | expr_node::Type::Case + | expr_node::Type::ConstantLookup | expr_node::Type::RoundDigit | expr_node::Type::Round | expr_node::Type::Ascii diff --git a/src/frontend/src/expr/type_inference/func.rs b/src/frontend/src/expr/type_inference/func.rs index 1ccbaa28e9da1..337c901a47eda 100644 --- a/src/frontend/src/expr/type_inference/func.rs +++ b/src/frontend/src/expr/type_inference/func.rs @@ -327,6 +327,21 @@ fn infer_type_for_special( .map(Some) .map_err(Into::into) } + ExprType::ConstantLookup => { + let len = inputs.len(); + align_types(inputs.iter_mut().enumerate().filter_map(|(i, e)| { + // This optimized `ConstantLookup` organize `inputs` as + // [dummy_expression] (cond, res) [else / fallback]? pairs. + // So we align exprs at even indices as well as the last one + // when length is odd. + match i != 0 && i.is_even() || i == len - 1 { + true => Some(e), + false => None, + } + })) + .map(Some) + .map_err(Into::into) + } ExprType::In => { align_types(inputs.iter_mut())?; Ok(Some(DataType::Boolean)) From d694af0851991a32ba052e1e112b6d57ef67d0b0 Mon Sep 17 00:00:00 2001 From: xfz <73645462+xuefengze@users.noreply.github.com> Date: Tue, 30 Jan 2024 10:11:47 +0800 Subject: [PATCH 07/14] feat(ci): add concurrency control for integration test (#14837) --- ci/scripts/gen-integration-test-yaml.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ci/scripts/gen-integration-test-yaml.py b/ci/scripts/gen-integration-test-yaml.py index 8451290a93c93..b8bc655b60bb1 100644 --- a/ci/scripts/gen-integration-test-yaml.py +++ b/ci/scripts/gen-integration-test-yaml.py @@ -51,6 +51,8 @@ def gen_pipeline_steps(): command: ci/scripts/integration-tests.sh -c {test_case} -f {test_format} timeout_in_minutes: 30 retry: *auto-retry + concurrency: 10 + concurrency_group: 'integration-test/run' plugins: - seek-oss/aws-sm#v2.3.1: env: From aeeb347dda5bc8b84d8025d4ce09a10fa2b6a455 Mon Sep 17 00:00:00 2001 From: "Zhanxiang (Patrick) Huang" Date: Tue, 30 Jan 2024 11:14:06 +0800 Subject: [PATCH 08/14] fix: fix compact task overlap check (#14856) --- .../compaction/picker/min_overlap_compaction_picker.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs index 23cee45b95ca6..8a76cc56f6b3c 100644 --- a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs @@ -377,9 +377,9 @@ impl NonOverlapSubLevelPicker { let start_idx = actual_sst_ids .iter() .position(|sst_id| sst_id == expected_sst_ids.first().unwrap()); - if start_idx.is_none() - || actual_sst_ids[start_idx.unwrap()..] != expected_sst_ids - { + if start_idx.map_or(true, |idx| { + actual_sst_ids[idx..idx + expected_sst_ids.len()] != expected_sst_ids + }) { // Print SstableInfo for `actual_sst_ids` let mut actual_sst_infos = String::new(); ssts.iter().for_each(|s| { From 9417409957dbdc047cc758b6652ae5134a68a9b4 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Tue, 30 Jan 2024 11:48:34 +0800 Subject: [PATCH 09/14] revert: feat(stream): make Project expr evaluation concurrent (#14558) (#14855) --- src/stream/src/executor/project.rs | 156 +++++++++-------------------- 1 file changed, 45 insertions(+), 111 deletions(-) diff --git a/src/stream/src/executor/project.rs b/src/stream/src/executor/project.rs index a5e73705f8ec4..efc381560e75d 100644 --- a/src/stream/src/executor/project.rs +++ b/src/stream/src/executor/project.rs @@ -14,8 +14,6 @@ use std::fmt::{Debug, Formatter}; -use auto_enums::auto_enum; -use futures::stream::Stream; use multimap::MultiMap; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; @@ -23,7 +21,6 @@ use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::ToOwnedDatum; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::NonStrictExpression; -use rw_futures_util::{RwFutureExt, RwTryStreamExt}; use super::*; @@ -33,8 +30,6 @@ use super::*; pub struct ProjectExecutor { input: BoxedExecutor, inner: Inner, - /// The mutable parts of inner fields. - vars: ExecutionVars, } struct Inner { @@ -48,17 +43,14 @@ struct Inner { watermark_derivations: MultiMap, /// Indices of nondecreasing expressions in the expression list. nondecreasing_expr_indices: Vec, + /// Last seen values of nondecreasing expressions, buffered to periodically produce watermarks. + last_nondec_expr_values: Vec>, /// the selectivity threshold which should be in `[0,1]`. for the chunk with selectivity less /// than the threshold, the Project executor will construct a new chunk before expr evaluation, materialize_selectivity_threshold: f64, } -struct ExecutionVars { - /// Last seen values of nondecreasing expressions, buffered to periodically produce watermarks. - last_nondec_expr_values: Vec>, -} - impl ProjectExecutor { #[allow(clippy::too_many_arguments)] pub fn new( @@ -79,10 +71,8 @@ impl ProjectExecutor { exprs, watermark_derivations, nondecreasing_expr_indices, - materialize_selectivity_threshold, - }, - vars: ExecutionVars { last_nondec_expr_values: vec![None; n_nondecreasing_exprs], + materialize_selectivity_threshold, }, } } @@ -110,7 +100,7 @@ impl Executor for ProjectExecutor { } fn execute(self: Box) -> BoxedMessageStream { - self.inner.execute(self.input, self.vars).boxed() + self.inner.execute(self.input).boxed() } } @@ -160,110 +150,54 @@ impl Inner { Ok(ret) } - fn execute( - self, - input: BoxedExecutor, - mut vars: ExecutionVars, - ) -> impl Stream { - let return_types: Vec<_> = self.exprs.iter().map(|expr| expr.return_type()).collect(); - - // Phase 1: only evaluating the expression, which can be concurrent. - - enum Phase1Item { - Chunk(Option), - Barrier(Barrier), - Watermark(Vec), - } - - let this = Arc::new(self); - - let this2 = this.clone(); - - let st = input.execute().map(move |msg| { - let this = this.clone(); + #[try_stream(ok = Message, error = StreamExecutorError)] + async fn execute(mut self, input: BoxedExecutor) { + #[for_await] + for msg in input.execute() { let msg = msg?; - let is_fence: bool; - #[auto_enum(Future)] - let fut = match msg { - Message::Chunk(chunk) => { - is_fence = false; - async move { - let new_chunk = this.map_filter_chunk(chunk).await?; - Ok(Phase1Item::Chunk(new_chunk)) as StreamExecutorResult<_> - } - } - Message::Watermark(watermark) => { - is_fence = false; - async move { - let watermarks = this.handle_watermark(watermark).await?; - Ok(Phase1Item::Watermark(watermarks)) + match msg { + Message::Watermark(w) => { + let watermarks = self.handle_watermark(w).await?; + for watermark in watermarks { + yield Message::Watermark(watermark) } } - Message::Barrier(barrier) => { - is_fence = true; - async { Ok(Phase1Item::Barrier(barrier)) } - } - }; - - let fut = fut.with_fence(is_fence); - - Ok(fut) as StreamExecutorResult<_> - }); - - // Make the phase 1 concurrent. - let st = st.try_buffered_with_fence(16); - - let this = this2; - - // Phase 2: Handle the watermark related logicals, and output them all. The phase is executed one by one. - #[try_stream] - async move { - #[for_await] - for msg in st { - let msg = msg?; - match msg { - Phase1Item::Watermark(watermarks) => { - for watermark in watermarks { - yield Message::Watermark(watermark) - } - } - Phase1Item::Chunk(new_chunk) => match new_chunk { - Some(new_chunk) => { - if !this.nondecreasing_expr_indices.is_empty() { - if let Some((_, first_visible_row)) = new_chunk.rows().next() { - // it's ok to use the first row here, just one chunk delay - first_visible_row - .project(&this.nondecreasing_expr_indices) - .iter() - .enumerate() - .for_each(|(idx, value)| { - vars.last_nondec_expr_values[idx] = - Some(value.to_owned_datum().expect( - "non-decreasing expression should never be NULL", - )); - }); - } + Message::Chunk(chunk) => match self.map_filter_chunk(chunk).await? { + Some(new_chunk) => { + if !self.nondecreasing_expr_indices.is_empty() { + if let Some((_, first_visible_row)) = new_chunk.rows().next() { + // it's ok to use the first row here, just one chunk delay + first_visible_row + .project(&self.nondecreasing_expr_indices) + .iter() + .enumerate() + .for_each(|(idx, value)| { + self.last_nondec_expr_values[idx] = + Some(value.to_owned_datum().expect( + "non-decreasing expression should never be NULL", + )); + }); } - yield Message::Chunk(new_chunk) } - None => continue, - }, - Phase1Item::Barrier(barrier) => { - for (&expr_idx, value) in this - .nondecreasing_expr_indices - .iter() - .zip_eq_fast(&mut vars.last_nondec_expr_values) - { - if let Some(value) = std::mem::take(value) { - yield Message::Watermark(Watermark::new( - expr_idx, - return_types[expr_idx].clone(), - value, - )) - } + yield Message::Chunk(new_chunk) + } + None => continue, + }, + barrier @ Message::Barrier(_) => { + for (&expr_idx, value) in self + .nondecreasing_expr_indices + .iter() + .zip_eq_fast(&mut self.last_nondec_expr_values) + { + if let Some(value) = std::mem::take(value) { + yield Message::Watermark(Watermark::new( + expr_idx, + self.exprs[expr_idx].return_type(), + value, + )) } - yield Message::Barrier(barrier); } + yield barrier; } } } From f4bae4b5d7369f85a0dd8fd04d48ab6a5e1c9c89 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 30 Jan 2024 12:01:29 +0800 Subject: [PATCH 10/14] chore(deps): Bump strum_macros from 0.25.3 to 0.26.1 (#14851) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 31 ++++++++++++++++++++++--------- src/cmd_all/Cargo.toml | 2 +- src/common/Cargo.toml | 2 +- src/connector/Cargo.toml | 2 +- 4 files changed, 25 insertions(+), 12 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8b2d83d9157c7..fffd5b98e4807 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -187,7 +187,7 @@ dependencies = [ "serde_json", "snap", "strum", - "strum_macros", + "strum_macros 0.25.3", "thiserror", "typed-builder 0.16.2", "uuid", @@ -212,7 +212,7 @@ dependencies = [ "serde", "serde_json", "strum", - "strum_macros", + "strum_macros 0.25.3", "thiserror", "typed-builder 0.18.0", "uuid", @@ -2222,7 +2222,7 @@ checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" dependencies = [ "crossterm 0.27.0", "strum", - "strum_macros", + "strum_macros 0.25.3", "unicode-width", ] @@ -3077,7 +3077,7 @@ dependencies = [ "datafusion-common", "sqlparser", "strum", - "strum_macros", + "strum_macros 0.25.3", ] [[package]] @@ -8689,7 +8689,7 @@ dependencies = [ "risingwave_rt", "shell-words", "strum", - "strum_macros", + "strum_macros 0.26.1", "task_stats_alloc", "tempfile", "tikv-jemallocator", @@ -8787,7 +8787,7 @@ dependencies = [ "speedate", "static_assertions", "strum", - "strum_macros", + "strum_macros 0.26.1", "sysinfo", "tempfile", "thiserror", @@ -9030,7 +9030,7 @@ dependencies = [ "serde_yaml", "simd-json", "strum", - "strum_macros", + "strum_macros 0.26.1", "syn 1.0.109", "tempfile", "thiserror", @@ -11171,7 +11171,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "242f76c50fd18cbf098607090ade73a08d39cfd84ea835f3796a2c855223b19b" dependencies = [ "strum", - "strum_macros", + "strum_macros 0.25.3", ] [[package]] @@ -11563,7 +11563,7 @@ version = "0.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" dependencies = [ - "strum_macros", + "strum_macros 0.25.3", ] [[package]] @@ -11579,6 +11579,19 @@ dependencies = [ "syn 2.0.48", ] +[[package]] +name = "strum_macros" +version = "0.26.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a3417fc93d76740d974a01654a09777cb500428cc874ca9f45edfe0c4d4cd18" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.48", +] + [[package]] name = "subprocess" version = "0.2.9" diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index e5aa9e3c68d04..f5a08e6c4b688 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -35,7 +35,7 @@ risingwave_meta_node = { workspace = true } risingwave_rt = { workspace = true } shell-words = "1.1.0" strum = "0.25" -strum_macros = "0.25" +strum_macros = "0.26" tempfile = "3" tikv-jemallocator = { workspace = true, features = [ "unprefixed_malloc_on_supported_platforms", diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 78de81f426bfe..30ecaa87a8d1a 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -93,7 +93,7 @@ smallbitset = "0.7.1" speedate = "0.13.0" static_assertions = "1" strum = "0.25" -strum_macros = "0.25" +strum_macros = "0.26" sysinfo = { version = "0.30", default-features = false } thiserror = "1" thiserror-ext = { workspace = true } diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 2bf42bf38d81b..e8582d1f3e294 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -119,7 +119,7 @@ serde_json = "1" serde_with = { version = "3", features = ["json"] } simd-json = "0.13.3" strum = "0.25" -strum_macros = "0.25" +strum_macros = "0.26" tempfile = "3" thiserror = "1" thiserror-ext = { workspace = true } From 1ecfe4ce3297bcca5f8eb7e1a52e34be5d3d9b2e Mon Sep 17 00:00:00 2001 From: "Zhanxiang (Patrick) Huang" Date: Tue, 30 Jan 2024 13:41:54 +0800 Subject: [PATCH 11/14] chore: remove the unused mem_table_spill_threshold opt (#14839) --- src/cmd_all/src/standalone.rs | 1 - src/compute/src/lib.rs | 8 -------- 2 files changed, 9 deletions(-) diff --git a/src/cmd_all/src/standalone.rs b/src/cmd_all/src/standalone.rs index a51fb03120313..24b90ad613da4 100644 --- a/src/cmd_all/src/standalone.rs +++ b/src/cmd_all/src/standalone.rs @@ -300,7 +300,6 @@ mod test { connector_rpc_sink_payload_format: None, config_path: "src/config/test.toml", total_memory_bytes: 34359738368, - mem_table_spill_threshold: 4194304, parallelism: 10, role: Both, metrics_level: None, diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index 342448066dd06..b3125b76052a6 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -92,10 +92,6 @@ pub struct ComputeNodeOpts { #[clap(long, env = "RW_TOTAL_MEMORY_BYTES", default_value_t = default_total_memory_bytes())] pub total_memory_bytes: usize, - /// Spill threshold for mem table. - #[clap(long, env = "RW_MEM_TABLE_SPILL_THRESHOLD", default_value_t = default_mem_table_spill_threshold())] - pub mem_table_spill_threshold: usize, - /// The parallelism that the compute node will register to the scheduler of the meta service. #[clap(long, env = "RW_PARALLELISM", default_value_t = default_parallelism())] #[override_opts(if_absent, path = streaming.actor_runtime_worker_threads_num)] @@ -231,10 +227,6 @@ fn default_total_memory_bytes() -> usize { (system_memory_available_bytes() as f64 * DEFAULT_MEMORY_PROPORTION) as usize } -fn default_mem_table_spill_threshold() -> usize { - (4 << 20) as usize -} - fn default_parallelism() -> usize { total_cpu_available().ceil() as usize } From 4358a4ec92c61b3b3ac013651ca3f490c02f0ff3 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Tue, 30 Jan 2024 14:29:20 +0800 Subject: [PATCH 12/14] feat(expr): allow partial option arguments in functions (#14738) Signed-off-by: Runji Wang --- src/expr/impl/src/scalar/array_positions.rs | 14 ++++------ src/expr/impl/src/scalar/array_remove.rs | 8 ++---- src/expr/impl/src/scalar/array_replace.rs | 9 +++--- src/expr/impl/src/scalar/format_type.rs | 13 +++++---- src/expr/impl/src/scalar/string_to_array.rs | 18 ++++-------- src/expr/macro/src/gen.rs | 22 +++++++++++++-- src/expr/macro/src/lib.rs | 21 +++++++------- src/expr/macro/src/parse.rs | 31 +++++++++------------ 8 files changed, 67 insertions(+), 69 deletions(-) diff --git a/src/expr/impl/src/scalar/array_positions.rs b/src/expr/impl/src/scalar/array_positions.rs index 5218dbce6780f..cbae53c001439 100644 --- a/src/expr/impl/src/scalar/array_positions.rs +++ b/src/expr/impl/src/scalar/array_positions.rs @@ -66,10 +66,7 @@ use risingwave_expr::{function, ExprError, Result}; /// 2 /// ``` #[function("array_position(anyarray, any) -> int4")] -fn array_position( - array: Option>, - element: Option>, -) -> Result> { +fn array_position(array: ListRef<'_>, element: Option>) -> Result> { array_position_common(array, element, 0) } @@ -98,7 +95,7 @@ fn array_position( /// ``` #[function("array_position(anyarray, any, int4) -> int4")] fn array_position_start( - array: Option>, + array: ListRef<'_>, element: Option>, start: Option, ) -> Result> { @@ -115,16 +112,15 @@ fn array_position_start( } fn array_position_common( - array: Option>, + array: ListRef<'_>, element: Option>, skip: usize, ) -> Result> { - let Some(left) = array else { return Ok(None) }; - if i32::try_from(left.len()).is_err() { + if i32::try_from(array.len()).is_err() { return Err(ExprError::CastOutOfRange("invalid array length")); } - Ok(left + Ok(array .iter() .skip(skip) .position(|item| item == element) diff --git a/src/expr/impl/src/scalar/array_remove.rs b/src/expr/impl/src/scalar/array_remove.rs index 4608d52602c11..32cfbbae96538 100644 --- a/src/expr/impl/src/scalar/array_remove.rs +++ b/src/expr/impl/src/scalar/array_remove.rs @@ -67,10 +67,6 @@ use risingwave_expr::function; /// select array_remove(ARRAY[array[1],array[2],array[3],array[2],null], array[true]); /// ``` #[function("array_remove(anyarray, any) -> anyarray")] -fn array_remove(array: Option>, elem: Option>) -> Option { - let array = array?; - Some(ListValue::from_datum_iter( - &array.data_type(), - array.iter().filter(|x| x != &elem), - )) +fn array_remove(array: ListRef<'_>, elem: Option>) -> ListValue { + ListValue::from_datum_iter(&array.data_type(), array.iter().filter(|x| x != &elem)) } diff --git a/src/expr/impl/src/scalar/array_replace.rs b/src/expr/impl/src/scalar/array_replace.rs index 5a9014cd445df..637062d946047 100644 --- a/src/expr/impl/src/scalar/array_replace.rs +++ b/src/expr/impl/src/scalar/array_replace.rs @@ -56,16 +56,15 @@ use risingwave_expr::function; /// ``` #[function("array_replace(anyarray, any, any) -> anyarray")] fn array_replace( - array: Option>, + array: ListRef<'_>, elem_from: Option>, elem_to: Option>, -) -> Option { - let array = array?; - Some(ListValue::from_datum_iter( +) -> ListValue { + ListValue::from_datum_iter( &array.data_type(), array.iter().map(|val| match val == elem_from { true => elem_to, false => val, }), - )) + ) } diff --git a/src/expr/impl/src/scalar/format_type.rs b/src/expr/impl/src/scalar/format_type.rs index ffc3eb7f17fb8..6384d8fc76966 100644 --- a/src/expr/impl/src/scalar/format_type.rs +++ b/src/expr/impl/src/scalar/format_type.rs @@ -12,17 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::fmt::Write; + use risingwave_common::types::DataType; use risingwave_expr::function; #[function("format_type(int4, int4) -> varchar")] -pub fn format_type(oid: Option, _typemod: Option) -> Option> { +pub fn format_type(oid: i32, _typemod: Option, writer: &mut impl Write) { // since we don't support type modifier, ignore it. - oid.map(|i| { - DataType::from_oid(i) - .map(|dt| format!("{}", dt).into_boxed_str()) - .unwrap_or("???".into()) - }) + match DataType::from_oid(oid) { + Ok(dt) => write!(writer, "{}", dt).unwrap(), + Err(_) => write!(writer, "???").unwrap(), + } } #[cfg(test)] diff --git a/src/expr/impl/src/scalar/string_to_array.rs b/src/expr/impl/src/scalar/string_to_array.rs index 0828b730d16fb..ebbc6fadd3d4a 100644 --- a/src/expr/impl/src/scalar/string_to_array.rs +++ b/src/expr/impl/src/scalar/string_to_array.rs @@ -34,25 +34,19 @@ fn string_to_array_inner<'a>(s: &'a str, sep: Option<&'a str>) -> impl Iterator< // Use cases shown in `e2e_test/batch/functions/string_to_array.slt.part` #[function("string_to_array(varchar, varchar) -> varchar[]")] -pub fn string_to_array2(s: Option<&str>, sep: Option<&str>) -> Option { - Some(ListValue::new( - string_to_array_inner(s?, sep).collect::().into(), - )) +pub fn string_to_array2(s: &str, sep: Option<&str>) -> ListValue { + ListValue::new(string_to_array_inner(s, sep).collect::().into()) } #[function("string_to_array(varchar, varchar, varchar) -> varchar[]")] -pub fn string_to_array3( - s: Option<&str>, - sep: Option<&str>, - null: Option<&str>, -) -> Option { +pub fn string_to_array3(s: &str, sep: Option<&str>, null: Option<&str>) -> ListValue { let Some(null) = null else { return string_to_array2(s, sep); }; - Some(ListValue::new( - string_to_array_inner(s?, sep) + ListValue::new( + string_to_array_inner(s, sep) .map(|x| if x == null { None } else { Some(x) }) .collect::() .into(), - )) + ) } diff --git a/src/expr/macro/src/gen.rs b/src/expr/macro/src/gen.rs index 28dd14c315f16..b7a8cd6424503 100644 --- a/src/expr/macro/src/gen.rs +++ b/src/expr/macro/src/gen.rs @@ -341,13 +341,31 @@ impl FunctionAttr { }; // if user function accepts non-option arguments, we assume the function // returns null on null input, so we need to unwrap the inputs before calling. - if !user_fn.arg_option { + if self.prebuild.is_some() { output = quote! { match (#(#inputs,)*) { (#(Some(#inputs),)*) => #output, _ => None, } }; + } else { + #[allow(clippy::disallowed_methods)] // allow zip + let some_inputs = inputs + .iter() + .zip(user_fn.args_option.iter()) + .map(|(input, opt)| { + if *opt { + quote! { #input } + } else { + quote! { Some(#input) } + } + }); + output = quote! { + match (#(#inputs,)*) { + (#(#some_inputs,)*) => #output, + _ => None, + } + }; }; // now the `output` is: Option let append_output = match user_fn.write { @@ -728,7 +746,7 @@ impl FunctionAttr { ReturnTypeKind::Result => quote! { Some(#next_state?) }, ReturnTypeKind::ResultOption => quote! { #next_state? }, }; - if !user_fn.accumulate().arg_option { + if user_fn.accumulate().args_option.iter().all(|b| !b) { match self.args.len() { 0 => { next_state = quote! { diff --git a/src/expr/macro/src/lib.rs b/src/expr/macro/src/lib.rs index 63e61dc736c0e..ceaef0d6d2406 100644 --- a/src/expr/macro/src/lib.rs +++ b/src/expr/macro/src/lib.rs @@ -187,11 +187,10 @@ mod utils; /// /// ```ignore /// #[function("trim_array(anyarray, int32) -> anyarray")] -/// fn trim_array(array: Option>, n: Option) -> ListValue {...} +/// fn trim_array(array: ListRef<'_>, n: Option) -> ListValue {...} /// ``` /// -/// Note that we currently only support all arguments being either `Option` or non-`Option`. Mixed -/// cases are not supported. +/// This function will be called when `n` is null, but not when `array` is null. /// /// ## Return Value /// @@ -278,11 +277,11 @@ mod utils; /// } /// ``` /// -/// The `prebuild` argument can be specified, and its value is a Rust expression used to construct a -/// new variable from the input arguments of the function. Here `$1`, `$2` represent the second and -/// third arguments of the function (indexed from 0), and their types are `&str`. In the Rust -/// function signature, these positions of parameters will be omitted, replaced by an extra new -/// variable at the end. +/// The `prebuild` argument can be specified, and its value is a Rust expression `Type::method(...)` +/// used to construct a new variable of `Type` from the input arguments of the function. +/// Here `$1`, `$2` represent the second and third arguments of the function (indexed from 0), +/// and their types are `&str`. In the Rust function signature, these positions of parameters will +/// be omitted, replaced by an extra new variable at the end. /// /// This macro generates two versions of the function. If all the input parameters that `prebuild` /// depends on are constants, it will precompute them during the build function. Otherwise, it will @@ -522,8 +521,8 @@ struct UserFunctionAttr { write: bool, /// Whether the last argument type is `retract: bool`. retract: bool, - /// The argument type are `Option`s. - arg_option: bool, + /// Whether each argument type is `Option`. + args_option: Vec, /// If the first argument type is `&mut T`, then `Some(T)`. first_mut_ref_arg: Option, /// The return type kind. @@ -610,7 +609,7 @@ impl UserFunctionAttr { !self.async_ && !self.write && !self.context - && !self.arg_option + && self.args_option.iter().all(|b| !b) && self.return_type_kind == ReturnTypeKind::T } } diff --git a/src/expr/macro/src/parse.rs b/src/expr/macro/src/parse.rs index af8eb38e543b6..d5b1ddf57fffb 100644 --- a/src/expr/macro/src/parse.rs +++ b/src/expr/macro/src/parse.rs @@ -124,7 +124,7 @@ impl From<&syn::Signature> for UserFunctionAttr { write: sig.inputs.iter().any(arg_is_write), context: sig.inputs.iter().any(arg_is_context), retract: last_arg_is_retract(sig), - arg_option: args_contain_option(sig), + args_option: sig.inputs.iter().map(arg_is_option).collect(), first_mut_ref_arg: first_mut_ref_arg(sig), return_type_kind, iterator_item_kind, @@ -224,23 +224,18 @@ fn last_arg_is_retract(sig: &syn::Signature) -> bool { pat.ident.to_string().contains("retract") } -/// Check if any argument is `Option`. -fn args_contain_option(sig: &syn::Signature) -> bool { - for arg in &sig.inputs { - let syn::FnArg::Typed(arg) = arg else { - continue; - }; - let syn::Type::Path(path) = arg.ty.as_ref() else { - continue; - }; - let Some(seg) = path.path.segments.last() else { - continue; - }; - if seg.ident == "Option" { - return true; - } - } - false +/// Check if the argument is `Option`. +fn arg_is_option(arg: &syn::FnArg) -> bool { + let syn::FnArg::Typed(arg) = arg else { + return false; + }; + let syn::Type::Path(path) = arg.ty.as_ref() else { + return false; + }; + let Some(seg) = path.path.segments.last() else { + return false; + }; + seg.ident == "Option" } /// Returns `T` if the first argument (except `self`) is `&mut T`. From 1e2605405fab48815aac4856be5d9a4f64824471 Mon Sep 17 00:00:00 2001 From: Yingjun Wu Date: Mon, 29 Jan 2024 23:49:52 -0800 Subject: [PATCH 13/14] doc: Update README.md (#14867) --- README.md | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/README.md b/README.md index e1cd266da5552..44443cfab8282 100644 --- a/README.md +++ b/README.md @@ -9,7 +9,7 @@
-### 🌊Reimagine Stream Processing. +### 🌊 Reimagine Stream Processing.
@@ -40,12 +40,6 @@ > Slack - - Docker - -RisingWave is a distributed SQL streaming database engineered to provide the simplest and most cost-efficient approach for processing, analyzing, and managing real-time event streaming data. +RisingWave is a Postgres-compatible streaming database engineered to provide the simplest and most cost-efficient approach for processing, analyzing, and managing real-time event streaming data. ![RisingWave](https://github.com/risingwavelabs/risingwave-docs/blob/main/docs/images/new_archi_grey.png) @@ -129,8 +123,9 @@ Within your data stack, RisingWave can assist with: * Processing and transforming event streaming data in real time * Offloading event-driven queries (e.g., materialized views, triggers) from operational databases * Performing real-time ETL (Extract, Transform, Load) +* Supporting real-time feature stores -RisingWave is extensively utilized in real-time applications such as monitoring, alerting, dashboard reporting, ML feature engineering, among others. It has already been adopted in fields such as financial trading, manufacturing, new media, logistics, gaming, and more. Check out [customer stories](https://www.risingwave.com/use-cases/). +RisingWave is extensively utilized in real-time applications such as monitoring, alerting, dashboard reporting, machine learning, among others. It has already been adopted in fields such as financial trading, manufacturing, new media, logistics, gaming, and more. Check out [customer stories](https://www.risingwave.com/use-cases/). ## Community From 63e548538b9def0a36f48e4bc7fdb9b58caf067e Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 30 Jan 2024 16:00:57 +0800 Subject: [PATCH 14/14] feat(stream): support row count for arrangement backfill (#14836) Co-authored-by: August --- risedev.yml | 2 +- .../executor/backfill/arrangement_backfill.rs | 18 +- src/stream/src/executor/backfill/utils.rs | 171 ++++++++++++++---- src/tests/simulation/src/cluster.rs | 2 + .../tests/integration_tests/backfill_tests.rs | 58 +++++- .../tests/integration_tests/main.rs | 2 + .../recovery/background_ddl.rs | 39 +--- .../tests/integration_tests/utils.rs | 51 ++++++ 8 files changed, 260 insertions(+), 83 deletions(-) create mode 100644 src/tests/simulation/tests/integration_tests/utils.rs diff --git a/risedev.yml b/risedev.yml index 38ed00e15fc63..93c7c0c1e90d0 100644 --- a/risedev.yml +++ b/risedev.yml @@ -570,7 +570,7 @@ profile: steps: - use: minio api-requests-max: 30 - api-requests-deadline: 2s + api-requests-deadline: 3s - use: etcd unsafe-no-fsync: true - use: meta-node diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index a7d2ab5c863eb..76f9271862e29 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -185,7 +185,7 @@ where let mut snapshot_read_epoch; // Keep track of rows from the snapshot. - let mut total_snapshot_processed_rows: u64 = 0; + let mut total_snapshot_processed_rows: u64 = backfill_state.get_snapshot_row_count(); // Arrangement Backfill Algorithm: // @@ -278,9 +278,8 @@ where // mark. for chunk in upstream_chunk_buffer.drain(..) { let chunk_cardinality = chunk.cardinality() as u64; - cur_barrier_snapshot_processed_rows += + cur_barrier_upstream_processed_rows += chunk_cardinality; - total_snapshot_processed_rows += chunk_cardinality; yield Message::Chunk(mapping_chunk( chunk, &self.output_indices, @@ -290,6 +289,8 @@ where break 'backfill_loop; } Some((vnode, chunk)) => { + let chunk_cardinality = chunk.cardinality() as u64; + // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. @@ -298,9 +299,9 @@ where &chunk, &pk_in_output_indices, &mut backfill_state, + chunk_cardinality, )?; - let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; total_snapshot_processed_rows += chunk_cardinality; let chunk = Message::Chunk(mapping_chunk( @@ -354,6 +355,7 @@ where }) })) { if let Some(chunk) = chunk { + let chunk_cardinality = chunk.cardinality() as u64; // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. @@ -362,9 +364,9 @@ where &chunk, &pk_in_output_indices, &mut backfill_state, + chunk_cardinality, )?; - let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += chunk_cardinality; total_snapshot_processed_rows += chunk_cardinality; yield Message::Chunk(mapping_chunk(chunk, &self.output_indices)); @@ -585,8 +587,10 @@ where let backfill_progress = backfill_state.get_progress(&vnode)?; let current_pos = match backfill_progress { BackfillProgressPerVnode::NotStarted => None, - BackfillProgressPerVnode::Completed(current_pos) - | BackfillProgressPerVnode::InProgress(current_pos) => Some(current_pos.clone()), + BackfillProgressPerVnode::Completed { current_pos, .. } + | BackfillProgressPerVnode::InProgress { current_pos, .. } => { + Some(current_pos.clone()) + } }; let range_bounds = compute_bounds(upstream_table.pk_indices(), current_pos.clone()); diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 1a19a3fe201fc..8937d52607748 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -90,12 +90,24 @@ impl BackfillState { &mut self, vnode: VirtualNode, new_pos: OwnedRow, + snapshot_row_count_delta: u64, ) -> StreamExecutorResult<()> { let state = self.get_current_state(&vnode); - let new_state = BackfillProgressPerVnode::InProgress(new_pos); match state { - BackfillProgressPerVnode::NotStarted => *state = new_state, - BackfillProgressPerVnode::InProgress(_current_pos) => *state = new_state, + BackfillProgressPerVnode::NotStarted => { + *state = BackfillProgressPerVnode::InProgress { + current_pos: new_pos, + snapshot_row_count: snapshot_row_count_delta, + }; + } + BackfillProgressPerVnode::InProgress { + snapshot_row_count, .. + } => { + *state = BackfillProgressPerVnode::InProgress { + current_pos: new_pos, + snapshot_row_count: *snapshot_row_count + snapshot_row_count_delta, + }; + } BackfillProgressPerVnode::Completed { .. } => unreachable!(), } Ok(()) @@ -104,14 +116,20 @@ impl BackfillState { pub(crate) fn finish_progress(&mut self, vnode: VirtualNode, pos_len: usize) { let finished_placeholder_position = construct_initial_finished_state(pos_len); let current_state = self.get_current_state(&vnode); - let new_pos = match current_state { - BackfillProgressPerVnode::NotStarted => finished_placeholder_position, - BackfillProgressPerVnode::InProgress(current_pos) => current_pos.clone(), + let (new_pos, snapshot_row_count) = match current_state { + BackfillProgressPerVnode::NotStarted => (finished_placeholder_position, 0), + BackfillProgressPerVnode::InProgress { + current_pos, + snapshot_row_count, + } => (current_pos.clone(), *snapshot_row_count), BackfillProgressPerVnode::Completed { .. } => { return; } }; - *current_state = BackfillProgressPerVnode::Completed(new_pos); + *current_state = BackfillProgressPerVnode::Completed { + current_pos: new_pos, + snapshot_row_count, + }; } /// Return state to be committed. @@ -119,42 +137,56 @@ impl BackfillState { let new_state = self.inner.get(vnode).unwrap().current_state().clone(); let new_encoded_state = match new_state { BackfillProgressPerVnode::NotStarted => unreachable!(), - BackfillProgressPerVnode::InProgress(current_pos) => { + BackfillProgressPerVnode::InProgress { + current_pos, + snapshot_row_count, + } => { let mut encoded_state = vec![None; current_pos.len() + METADATA_STATE_LEN]; encoded_state[0] = Some(vnode.to_scalar().into()); encoded_state[1..current_pos.len() + 1].clone_from_slice(current_pos.as_inner()); encoded_state[current_pos.len() + 1] = Some(false.into()); - encoded_state[current_pos.len() + 2] = Some(0i64.into()); + encoded_state[current_pos.len() + 2] = Some((snapshot_row_count as i64).into()); encoded_state } - BackfillProgressPerVnode::Completed(current_pos) => { + BackfillProgressPerVnode::Completed { + current_pos, + snapshot_row_count, + } => { let mut encoded_state = vec![None; current_pos.len() + METADATA_STATE_LEN]; encoded_state[0] = Some(vnode.to_scalar().into()); encoded_state[1..current_pos.len() + 1].clone_from_slice(current_pos.as_inner()); encoded_state[current_pos.len() + 1] = Some(true.into()); - encoded_state[current_pos.len() + 2] = Some(0i64.into()); + encoded_state[current_pos.len() + 2] = Some((snapshot_row_count as i64).into()); encoded_state } }; let old_state = self.inner.get(vnode).unwrap().committed_state().clone(); let old_encoded_state = match old_state { BackfillProgressPerVnode::NotStarted => None, - BackfillProgressPerVnode::InProgress(committed_pos) => { + BackfillProgressPerVnode::InProgress { + current_pos, + snapshot_row_count, + } => { + let committed_pos = current_pos; let mut encoded_state = vec![None; committed_pos.len() + METADATA_STATE_LEN]; encoded_state[0] = Some(vnode.to_scalar().into()); encoded_state[1..committed_pos.len() + 1] .clone_from_slice(committed_pos.as_inner()); encoded_state[committed_pos.len() + 1] = Some(false.into()); - encoded_state[committed_pos.len() + 2] = Some(0i64.into()); + encoded_state[committed_pos.len() + 2] = Some((snapshot_row_count as i64).into()); Some(encoded_state) } - BackfillProgressPerVnode::Completed(committed_pos) => { + BackfillProgressPerVnode::Completed { + current_pos, + snapshot_row_count, + } => { + let committed_pos = current_pos; let mut encoded_state = vec![None; committed_pos.len() + METADATA_STATE_LEN]; encoded_state[0] = Some(vnode.to_scalar().into()); encoded_state[1..committed_pos.len() + 1] .clone_from_slice(committed_pos.as_inner()); encoded_state[committed_pos.len() + 1] = Some(true.into()); - encoded_state[committed_pos.len() + 2] = Some(0i64.into()); + encoded_state[committed_pos.len() + 2] = Some((snapshot_row_count as i64).into()); Some(encoded_state) } }; @@ -167,8 +199,8 @@ impl BackfillState { let state = self.inner.get(vnode).unwrap(); match state.current_state() { // If current state and committed state are the same, we don't need to commit. - s @ BackfillProgressPerVnode::InProgress(_current_pos) - | s @ BackfillProgressPerVnode::Completed(_current_pos) => s != state.committed_state(), + s @ BackfillProgressPerVnode::InProgress { .. } + | s @ BackfillProgressPerVnode::Completed { .. } => s != state.committed_state(), BackfillProgressPerVnode::NotStarted => false, } } @@ -181,10 +213,18 @@ impl BackfillState { assert!(matches!( current_state, - BackfillProgressPerVnode::InProgress(_) | BackfillProgressPerVnode::Completed(_) + BackfillProgressPerVnode::InProgress { .. } + | BackfillProgressPerVnode::Completed { .. } )); *committed_state = current_state.clone(); } + + pub(crate) fn get_snapshot_row_count(&self) -> u64 { + self.inner + .values() + .map(|p| p.get_snapshot_row_count()) + .sum() + } } #[derive(Clone, Debug, PartialEq, Eq)] @@ -211,6 +251,10 @@ impl BackfillStatePerVnode { pub(crate) fn current_state(&self) -> &BackfillProgressPerVnode { &self.current_state } + + pub(crate) fn get_snapshot_row_count(&self) -> u64 { + self.current_state().get_snapshot_row_count() + } } impl From> for BackfillState { @@ -227,8 +271,32 @@ impl From> for BackfillState { pub enum BackfillProgressPerVnode { /// no entry exists for a vnode, or on initialization of the executor. NotStarted, - InProgress(OwnedRow), - Completed(OwnedRow), + InProgress { + /// The current snapshot offset + current_pos: OwnedRow, + /// Number of snapshot records read for this vnode. + snapshot_row_count: u64, + }, + Completed { + /// The current snapshot offset + current_pos: OwnedRow, + /// Number of snapshot records read for this vnode. + snapshot_row_count: u64, + }, +} + +impl BackfillProgressPerVnode { + fn get_snapshot_row_count(&self) -> u64 { + match self { + BackfillProgressPerVnode::NotStarted => 0, + BackfillProgressPerVnode::InProgress { + snapshot_row_count, .. + } + | BackfillProgressPerVnode::Completed { + snapshot_row_count, .. + } => *snapshot_row_count, + } + } } pub(crate) fn mark_chunk( @@ -280,11 +348,11 @@ pub(crate) fn mark_chunk_ref_by_vnode( let vnode = VirtualNode::compute_row(row, pk_in_output_indices); let v = match backfill_state.get_progress(&vnode)? { // We want to just forward the row, if the vnode has finished backfill. - BackfillProgressPerVnode::Completed(_) => true, + BackfillProgressPerVnode::Completed { .. } => true, // If not started, no need to forward. BackfillProgressPerVnode::NotStarted => false, // If in progress, we need to check row <= current_pos. - BackfillProgressPerVnode::InProgress(current_pos) => { + BackfillProgressPerVnode::InProgress { current_pos, .. } => { let lhs = row.project(pk_in_output_indices); let rhs = current_pos; let order = cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()); @@ -402,45 +470,71 @@ pub(crate) fn mapping_message(msg: Message, upstream_indices: &[usize]) -> Optio } /// Recovers progress per vnode, so we know which to backfill. +/// See how it decodes the state with the inline comments. pub(crate) async fn get_progress_per_vnode( state_table: &StateTableInner, ) -> StreamExecutorResult> { debug_assert!(!state_table.vnodes().is_empty()); let vnodes = state_table.vnodes().iter_vnodes(); let mut result = Vec::with_capacity(state_table.vnodes().len()); + // 1. Get the vnode keys, so we can get the state per vnode. let vnode_keys = vnodes.map(|vnode| { let datum: [Datum; 1] = [Some(vnode.to_scalar().into())]; datum }); let tasks = vnode_keys.map(|vnode_key| state_table.get_row(vnode_key)); - let states_for_vnode_keys = try_join_all(tasks).await?; - for (vnode, state_for_vnode_key) in state_table + // 2. Fetch the state for each vnode. + // It should have the following schema, it should not contain vnode: + // | pk | `backfill_finished` | `row_count` | + let state_for_vnodes = try_join_all(tasks).await?; + for (vnode, state_for_vnode) in state_table .vnodes() .iter_vnodes() - .zip_eq_debug(states_for_vnode_keys) + .zip_eq_debug(state_for_vnodes) { - // NOTE(kwannoel): state_for_vnode_key does not include the vnode prefix. - let backfill_progress = match state_for_vnode_key { + let backfill_progress = match state_for_vnode { + // There's some state, means there was progress made. It's either finished / in progress. Some(row) => { + // 3. Decode the `snapshot_row_count`. Decode from the back, since + // pk is variable length. + let snapshot_row_count = row.as_inner().get(row.len() - 1).unwrap(); + let snapshot_row_count = (*snapshot_row_count.as_ref().unwrap().as_int64()) as u64; + + // 4. Decode the `is_finished` flag (whether backfill has finished). + // Decode from the back, since pk is variable length. let vnode_is_finished = row.as_inner().get(row.len() - 2).unwrap(); let vnode_is_finished = vnode_is_finished.as_ref().unwrap(); - // Only the current pos should be contained in the in-memory backfill state. - // Row count will be added later. + // 5. Decode the `current_pos`. let current_pos = row.as_inner().get(..row.len() - 2).unwrap(); let current_pos = current_pos.into_owned_row(); + + // 6. Construct the in-memory state per vnode, based on the decoded state. if *vnode_is_finished.as_bool() { BackfillStatePerVnode::new( - BackfillProgressPerVnode::Completed(current_pos.clone()), - BackfillProgressPerVnode::Completed(current_pos), + BackfillProgressPerVnode::Completed { + current_pos: current_pos.clone(), + snapshot_row_count, + }, + BackfillProgressPerVnode::Completed { + current_pos, + snapshot_row_count, + }, ) } else { BackfillStatePerVnode::new( - BackfillProgressPerVnode::InProgress(current_pos.clone()), - BackfillProgressPerVnode::InProgress(current_pos), + BackfillProgressPerVnode::InProgress { + current_pos: current_pos.clone(), + snapshot_row_count, + }, + BackfillProgressPerVnode::InProgress { + current_pos, + snapshot_row_count, + }, ) } } + // No state, means no progress made. None => BackfillStatePerVnode::new( BackfillProgressPerVnode::NotStarted, BackfillProgressPerVnode::NotStarted, @@ -512,10 +606,11 @@ pub(crate) fn update_pos_by_vnode( chunk: &StreamChunk, pk_in_output_indices: &[usize], backfill_state: &mut BackfillState, + snapshot_row_count_delta: u64, ) -> StreamExecutorResult<()> { let new_pos = get_new_pos(chunk, pk_in_output_indices); assert_eq!(new_pos.len(), pk_in_output_indices.len()); - backfill_state.update_progress(vnode, new_pos)?; + backfill_state.update_progress(vnode, new_pos, snapshot_row_count_delta)?; Ok(()) } @@ -604,7 +699,7 @@ where } /// Schema -/// | vnode | pk | `backfill_finished` | +/// | vnode | pk | `backfill_finished` | `row_count` | /// Persists the state per vnode based on `BackfillState`. /// We track the current committed state via `committed_progress` /// so we know whether we need to persist the state or not. @@ -614,12 +709,12 @@ where /// - Not persist to store at all. /// /// `InProgress`: -/// - Format: | vnode | pk | false | +/// - Format: | vnode | pk | false | `row_count` | /// - If change in current pos: Persist. /// - No change in current pos: Do not persist. /// /// Completed -/// - Format: | vnode | pk | true | +/// - Format: | vnode | pk | true | `row_count` | /// - If previous state is `InProgress` / `NotStarted`: Persist. /// - If previous state is Completed: Do not persist. /// TODO(kwannoel): we should check committed state to be all `finished` in the tests. @@ -693,7 +788,7 @@ pub(crate) async fn persist_state_per_vnode Result<()> { let upstream_task = tokio::spawn(async move { // The initial 100 records will take approx 3s // After that we start ingesting upstream records. - sleep(Duration::from_secs(3)); + sleep(Duration::from_secs(3)).await; for i in 101..=200 { session2 .run(format!("insert into t values ({})", i)) @@ -233,3 +234,54 @@ async fn test_backfill_backpressure() -> Result<()> { // distribution MUST also be single, and arrangement backfill should just use Simple. // TODO(kwannoel): Test arrangement backfill background recovery. +#[tokio::test] +async fn test_arrangement_backfill_progress() -> Result<()> { + let mut cluster = Cluster::start(Configuration::for_arrangement_backfill()).await?; + let mut session = cluster.start_session(); + + // Create base table + session.run("CREATE TABLE t (v1 int primary key)").await?; + + // Ingest data + session + .run("INSERT INTO t SELECT * FROM generate_series(1, 1000)") + .await?; + session.run("FLUSH;").await?; + + // Create arrangement backfill with rate limit + session.run("SET STREAMING_PARALLELISM=1").await?; + session.run("SET BACKGROUND_DDL=true").await?; + session.run("SET STREAMING_RATE_LIMIT=1").await?; + session + .run("CREATE MATERIALIZED VIEW m1 AS SELECT * FROM t") + .await?; + + // Verify arrangement backfill progress after 10s, it should be 1% at least. + sleep(Duration::from_secs(10)).await; + let progress = session + .run("SELECT progress FROM rw_catalog.rw_ddl_progress") + .await?; + let progress = progress.replace('%', ""); + let progress = progress.parse::().unwrap(); + assert!( + (1.0..2.0).contains(&progress), + "progress not within bounds {}", + progress + ); + + // Trigger recovery and test it again. + kill_cn_and_wait_recover(&cluster).await; + let prev_progress = progress; + let progress = session + .run("SELECT progress FROM rw_catalog.rw_ddl_progress") + .await?; + let progress = progress.replace('%', ""); + let progress = progress.parse::().unwrap(); + assert!( + (prev_progress - 0.5..prev_progress + 1.5).contains(&progress), + "progress not within bounds {}", + progress + ); + + Ok(()) +} diff --git a/src/tests/simulation/tests/integration_tests/main.rs b/src/tests/simulation/tests/integration_tests/main.rs index 43262cd7b52a2..475793a88b709 100644 --- a/src/tests/simulation/tests/integration_tests/main.rs +++ b/src/tests/simulation/tests/integration_tests/main.rs @@ -28,3 +28,5 @@ mod scale; mod sink; mod storage; mod throttle; + +mod utils; diff --git a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs index 7f6c2da07d64e..e7792d5930e03 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs @@ -16,9 +16,13 @@ use std::time::Duration; use anyhow::Result; use risingwave_common::error::anyhow_error; -use risingwave_simulation::cluster::{Cluster, Configuration, KillOpts, Session}; +use risingwave_simulation::cluster::{Cluster, Configuration, Session}; use tokio::time::sleep; +use crate::utils::{ + kill_cn_and_meta_and_wait_recover, kill_cn_and_wait_recover, kill_random_and_wait_recover, +}; + const CREATE_TABLE: &str = "CREATE TABLE t(v1 int);"; const DROP_TABLE: &str = "DROP TABLE t;"; const SEED_TABLE_500: &str = "INSERT INTO t SELECT generate_series FROM generate_series(1, 500);"; @@ -31,39 +35,6 @@ const CREATE_MV1: &str = "CREATE MATERIALIZED VIEW mv1 as SELECT * FROM t;"; const DROP_MV1: &str = "DROP MATERIALIZED VIEW mv1;"; const WAIT: &str = "WAIT;"; -async fn kill_cn_and_wait_recover(cluster: &Cluster) { - cluster - .kill_nodes(["compute-1", "compute-2", "compute-3"], 0) - .await; - sleep(Duration::from_secs(10)).await; -} - -async fn kill_cn_and_meta_and_wait_recover(cluster: &Cluster) { - cluster - .kill_nodes( - [ - "compute-1", - "compute-2", - "compute-3", - "meta-1", - "meta-2", - "meta-3", - ], - 0, - ) - .await; - sleep(Duration::from_secs(10)).await; -} - -async fn kill_random_and_wait_recover(cluster: &Cluster) { - // Kill it again - for _ in 0..3 { - sleep(Duration::from_secs(2)).await; - cluster.kill_node(&KillOpts::ALL_FAST).await; - } - sleep(Duration::from_secs(10)).await; -} - async fn cancel_stream_jobs(session: &mut Session) -> Result> { tracing::info!("finding streaming jobs to cancel"); let ids = session diff --git a/src/tests/simulation/tests/integration_tests/utils.rs b/src/tests/simulation/tests/integration_tests/utils.rs new file mode 100644 index 0000000000000..8f06d0acbea2f --- /dev/null +++ b/src/tests/simulation/tests/integration_tests/utils.rs @@ -0,0 +1,51 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::time::Duration; + +use risingwave_simulation::cluster::{Cluster, KillOpts}; +use tokio::time::sleep; + +pub(crate) async fn kill_cn_and_wait_recover(cluster: &Cluster) { + cluster + .kill_nodes(["compute-1", "compute-2", "compute-3"], 0) + .await; + sleep(Duration::from_secs(10)).await; +} + +pub(crate) async fn kill_cn_and_meta_and_wait_recover(cluster: &Cluster) { + cluster + .kill_nodes( + [ + "compute-1", + "compute-2", + "compute-3", + "meta-1", + "meta-2", + "meta-3", + ], + 0, + ) + .await; + sleep(Duration::from_secs(10)).await; +} + +pub(crate) async fn kill_random_and_wait_recover(cluster: &Cluster) { + // Kill it again + for _ in 0..3 { + sleep(Duration::from_secs(2)).await; + cluster.kill_node(&KillOpts::ALL_FAST).await; + } + sleep(Duration::from_secs(10)).await; +}