From 96984005348c67c1031444ce4f57f35735ce55d0 Mon Sep 17 00:00:00 2001 From: discord9 Date: Mon, 18 May 2026 10:40:27 +0800 Subject: [PATCH 1/4] feat: support defer_on_missing_source for pending flow creation Add `defer_on_missing_source` flow option that allows creating flows even when source tables do not yet exist. The flow enters a pending state and is automatically activated when source tables become available. Key changes: - New `FlowStatus::PendingSources` and fields in `FlowInfoValue` for unresolved source table names and last activation error - `defer_on_missing_source` create-time-only option: stripped from runtime/flownode `CreateRequest` but preserved in metadata for SQL round-trip (`SHOW CREATE FLOW`, `information_schema.flows`) - `CreateFlowProcedure` creates pending metadata when sources are missing and `defer_on_missing_source=true`; falls back to `FlowType::Batching` for missing-source flows - `PendingFlowReconcileManager` in meta-srv periodically checks pending flows and activates them when source tables resolve - `ActivatePendingFlowProcedure` handles activation: allocates peers, creates flows on flownodes, updates metadata, invalidates cache - `OR REPLACE` properly handles pending<->active transitions, including peer allocation and flownode flow teardown - `FlowMetadataAllocator::alloc_peers` for peer allocation at activation time - Validated flow options: only `defer_on_missing_source` allowed; unknown options rejected - Known issue: standalone mode does not support flownodes, so pending flow flush/sink behavior covered only in distributed sqlness; operator and meta unit tests cover activation logic Tests: - operator `determine_flow_type_for_source_state` (3 passed) - common-meta `create_flow` (19 passed) including replacement - common-meta `activate_flow` (4 passed) - meta-srv `flow` (11 passed) - sqlness: `flow_pending` covers create/replace/round-trip Signed-off-by: discord9 --- .../meta/src/cache/flow/table_flownode.rs | 6 +- src/common/meta/src/ddl.rs | 1 + src/common/meta/src/ddl/activate_flow.rs | 468 ++++++++++++++++++ src/common/meta/src/ddl/create_flow.rs | 140 +++++- .../meta/src/ddl/create_flow/metadata.rs | 29 +- src/common/meta/src/ddl/flow_meta.rs | 7 +- src/common/meta/src/ddl/tests.rs | 1 + .../meta/src/ddl/tests/activate_flow.rs | 329 ++++++++++++ src/common/meta/src/ddl/tests/create_flow.rs | 367 +++++++++++++- src/common/meta/src/ddl_manager.rs | 16 + src/common/meta/src/key/flow.rs | 13 + src/common/meta/src/key/flow/flow_info.rs | 72 ++- src/meta-srv/src/error.rs | 16 + src/meta-srv/src/flow.rs | 164 ++++++ src/meta-srv/src/lib.rs | 1 + src/meta-srv/src/metasrv.rs | 5 + src/meta-srv/src/metasrv/builder.rs | 11 + src/operator/src/statement/ddl.rs | 92 +++- .../common/flow/flow_pending.result | 148 ++++++ .../standalone/common/flow/flow_pending.sql | 83 ++++ 20 files changed, 1929 insertions(+), 40 deletions(-) create mode 100644 src/common/meta/src/ddl/activate_flow.rs create mode 100644 src/common/meta/src/ddl/tests/activate_flow.rs create mode 100644 src/meta-srv/src/flow.rs create mode 100644 tests/cases/standalone/common/flow/flow_pending.result create mode 100644 tests/cases/standalone/common/flow/flow_pending.sql diff --git a/src/common/meta/src/cache/flow/table_flownode.rs b/src/common/meta/src/cache/flow/table_flownode.rs index ebe3664202fa..8cd5d8b66233 100644 --- a/src/common/meta/src/cache/flow/table_flownode.rs +++ b/src/common/meta/src/cache/flow/table_flownode.rs @@ -210,7 +210,7 @@ mod tests { use crate::cache::flow::table_flownode::{FlowIdent, new_table_flownode_set_cache}; use crate::instruction::{CacheIdent, CreateFlow, DropFlow}; use crate::key::flow::FlowMetadataManager; - use crate::key::flow::flow_info::FlowInfoValue; + use crate::key::flow::flow_info::{FlowInfoValue, FlowStatus}; use crate::key::flow::flow_route::FlowRouteValue; use crate::kv_backend::memory::MemoryKvBackend; use crate::peer::Peer; @@ -242,11 +242,15 @@ mod tests { catalog_name: DEFAULT_CATALOG_NAME.to_string(), query_context: None, flow_name: "my_flow".to_string(), + all_source_table_names: vec![], + unresolved_source_table_names: vec![], raw_sql: "sql".to_string(), expire_after: Some(300), eval_interval_secs: None, comment: "comment".to_string(), options: Default::default(), + status: FlowStatus::Active, + last_activation_error: None, created_time: chrono::Utc::now(), updated_time: chrono::Utc::now(), }, diff --git a/src/common/meta/src/ddl.rs b/src/common/meta/src/ddl.rs index 8fc647433a3c..42fd8ffe7669 100644 --- a/src/common/meta/src/ddl.rs +++ b/src/common/meta/src/ddl.rs @@ -28,6 +28,7 @@ use crate::node_manager::NodeManagerRef; use crate::region_keeper::MemoryRegionKeeperRef; use crate::region_registry::LeaderRegionRegistryRef; +pub mod activate_flow; pub mod allocator; pub mod alter_database; pub mod alter_logical_tables; diff --git a/src/common/meta/src/ddl/activate_flow.rs b/src/common/meta/src/ddl/activate_flow.rs new file mode 100644 index 000000000000..86a1f5fec3f9 --- /dev/null +++ b/src/common/meta/src/ddl/activate_flow.rs @@ -0,0 +1,468 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::BTreeMap; + +use api::v1::ExpireAfter; +use api::v1::flow::flow_request::Body as PbFlowRequest; +use api::v1::flow::{CreateRequest, FlowRequest, FlowRequestHeader}; +use async_trait::async_trait; +use common_procedure::error::{FromJsonSnafu, ToJsonSnafu}; +use common_procedure::{ + Context as ProcedureContext, LockKey, Procedure, Result as ProcedureResult, Status, +}; +use common_telemetry::tracing_context::TracingContext; +use futures::future::join_all; +use itertools::Itertools; +use serde::{Deserialize, Serialize}; +use snafu::{OptionExt, ResultExt}; +use table::metadata::TableId; +use table::table_name::TableName; + +use crate::cache_invalidator::Context; +use crate::ddl::DdlContext; +use crate::ddl::create_flow::{DEFER_ON_MISSING_SOURCE_KEY, FlowType}; +use crate::ddl::utils::{add_peer_context_if_needed, map_to_procedure_error}; +use crate::error::{self, Result}; +use crate::instruction::{CacheIdent, CreateFlow}; +use crate::key::flow::flow_info::{FlowInfoValue, FlowStatus}; +use crate::key::flow::flow_route::FlowRouteValue; +use crate::key::table_name::TableNameKey; +use crate::key::{DeserializedValueWithBytes, FlowId, FlowPartitionId}; +use crate::lock_key::{CatalogLock, FlowLock, FlowNameLock}; +use crate::metrics; +use crate::peer::Peer; + +/// Activates a pending flow after all source tables become available. +pub struct ActivatePendingFlowProcedure { + pub context: DdlContext, + pub data: ActivatePendingFlowData, +} + +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] +pub enum ActivatePendingFlowState { + Prepare, + CreateFlows, + UpdateMetadata, + InvalidateFlowCache, +} + +#[derive(Debug, Serialize, Deserialize)] +pub struct ActivatePendingFlowData { + pub(crate) state: ActivatePendingFlowState, + pub(crate) flow_id: FlowId, + pub(crate) catalog_name: String, + #[serde(default)] + pub(crate) peers: Vec, + #[serde(default)] + pub(crate) resolved_table_ids: Vec, + pub(crate) prev_flow_info_value: Option>, +} + +pub struct PendingFlowResolution { + resolved_table_ids: Vec, + unresolved_source_table_names: Vec, +} + +impl ActivatePendingFlowProcedure { + pub const TYPE_NAME: &'static str = "metasrv-procedure::ActivatePendingFlow"; + + pub fn new(flow_id: FlowId, catalog_name: String, context: DdlContext) -> Self { + Self { + context, + data: ActivatePendingFlowData { + state: ActivatePendingFlowState::Prepare, + flow_id, + catalog_name, + peers: vec![], + resolved_table_ids: vec![], + prev_flow_info_value: None, + }, + } + } + + pub fn from_json(json: &str, context: DdlContext) -> ProcedureResult { + let data = serde_json::from_str(json).context(FromJsonSnafu)?; + Ok(Self { context, data }) + } + + async fn on_prepare(&mut self) -> Result { + let Some(current_flow_info) = self + .context + .flow_metadata_manager + .flow_info_manager() + .get_raw(self.data.flow_id) + .await? + else { + common_telemetry::debug!( + "Pending flow {} no longer exists during activation", + self.data.flow_id + ); + return Ok(Status::done()); + }; + + if current_flow_info.get_inner_ref().is_active() { + let routes = self + .context + .flow_metadata_manager + .flow_route_manager() + .routes(self.data.flow_id) + .await?; + self.data.peers = routes.into_iter().map(|(_, value)| value.peer).collect(); + self.data.resolved_table_ids = + current_flow_info.get_inner_ref().source_table_ids.clone(); + self.data.state = ActivatePendingFlowState::InvalidateFlowCache; + return Ok(Status::executing(true)); + } + + let resolution = + resolve_pending_flow_sources(&self.context, current_flow_info.get_inner_ref()).await?; + common_telemetry::debug!( + "Resolved pending flow {} source tables: {} resolved, {} unresolved", + self.data.flow_id, + resolution.resolved_table_ids.len(), + resolution.unresolved_source_table_names.len() + ); + if !resolution.unresolved_source_table_names.is_empty() { + update_pending_flow_metadata( + &self.context, + self.data.flow_id, + ¤t_flow_info, + resolution.resolved_table_ids, + resolution.unresolved_source_table_names, + None, + ) + .await?; + return Ok(Status::done()); + } + + self.data.peers = self.context.flow_metadata_allocator.alloc_peers(1).await?; + self.data.resolved_table_ids = resolution.resolved_table_ids; + self.data.prev_flow_info_value = Some(current_flow_info); + self.data.state = ActivatePendingFlowState::CreateFlows; + + Ok(Status::executing(true)) + } + + async fn on_create_flows(&mut self) -> Result { + let flow_info = + self.data + .prev_flow_info_value + .as_ref() + .context(error::UnexpectedSnafu { + err_msg: "missing previous flow info for activation", + })?; + let request = build_create_request( + self.data.flow_id, + flow_info.get_inner_ref(), + &self.data.resolved_table_ids, + ); + debug_assert_eq!( + self.data.resolved_table_ids.len(), + flow_info.get_inner_ref().all_source_table_names().len(), + "All source tables must be resolved before pending flow activation" + ); + create_flow_on_peers( + &self.context, + &self.data.peers, + request, + flow_info.get_inner_ref(), + ) + .await?; + self.data.state = ActivatePendingFlowState::UpdateMetadata; + Ok(Status::executing(true)) + } + + async fn on_update_metadata(&mut self) -> Result { + let current_flow_info = + self.data + .prev_flow_info_value + .as_ref() + .context(error::UnexpectedSnafu { + err_msg: "missing previous flow info for metadata update", + })?; + let (new_flow_info, flow_routes) = build_active_flow_info( + current_flow_info.get_inner_ref(), + &self.data.peers, + &self.data.resolved_table_ids, + ); + self.context + .flow_metadata_manager + .update_flow_metadata( + self.data.flow_id, + current_flow_info, + &new_flow_info, + flow_routes, + ) + .await?; + self.data.state = ActivatePendingFlowState::InvalidateFlowCache; + Ok(Status::executing(true)) + } + + async fn on_broadcast(&mut self) -> Result { + invalidate_flow_cache( + &self.context, + self.data.flow_id, + &self.data.resolved_table_ids, + &self.data.peers, + ) + .await + .map_err(error::Error::retry_later)?; + Ok(Status::done_with_output(self.data.flow_id)) + } +} + +#[async_trait] +impl Procedure for ActivatePendingFlowProcedure { + fn type_name(&self) -> &str { + Self::TYPE_NAME + } + + async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult { + let state = &self.data.state; + let _timer = metrics::METRIC_META_PROCEDURE_CREATE_FLOW + .with_label_values(&[match state { + ActivatePendingFlowState::Prepare => "activate_prepare", + ActivatePendingFlowState::CreateFlows => "activate_create_flows", + ActivatePendingFlowState::UpdateMetadata => "activate_update_metadata", + ActivatePendingFlowState::InvalidateFlowCache => "activate_invalidate_flow_cache", + }]) + .start_timer(); + + match self.data.state { + ActivatePendingFlowState::Prepare => self.on_prepare().await, + ActivatePendingFlowState::CreateFlows => self.on_create_flows().await, + ActivatePendingFlowState::UpdateMetadata => self.on_update_metadata().await, + ActivatePendingFlowState::InvalidateFlowCache => self.on_broadcast().await, + } + .map_err(map_to_procedure_error) + } + + fn dump(&self) -> ProcedureResult { + serde_json::to_string(&self.data).context(ToJsonSnafu) + } + + fn lock_key(&self) -> LockKey { + if let Some(flow_info) = &self.data.prev_flow_info_value { + LockKey::new(vec![ + CatalogLock::Read(&self.data.catalog_name).into(), + FlowNameLock::new( + flow_info.get_inner_ref().catalog_name(), + flow_info.get_inner_ref().flow_name(), + ) + .into(), + FlowLock::Write(self.data.flow_id).into(), + ]) + } else { + LockKey::new(vec![ + CatalogLock::Read(&self.data.catalog_name).into(), + FlowLock::Write(self.data.flow_id).into(), + ]) + } + } +} + +async fn resolve_pending_flow_sources( + context: &DdlContext, + flow_info: &FlowInfoValue, +) -> Result { + let keys = flow_info + .all_source_table_names() + .iter() + .map(|name| TableNameKey::new(&name.catalog_name, &name.schema_name, &name.table_name)) + .collect::>(); + let resolved_tables = context + .table_metadata_manager + .table_name_manager() + .batch_get(keys) + .await?; + + let mut resolved_table_ids = Vec::with_capacity(flow_info.all_source_table_names().len()); + let mut unresolved_source_table_names = Vec::new(); + for (name, table_id) in flow_info + .all_source_table_names() + .iter() + .zip(resolved_tables) + { + match table_id { + Some(table_id) => resolved_table_ids.push(table_id.table_id()), + None => unresolved_source_table_names.push(name.clone()), + } + } + + Ok(PendingFlowResolution { + resolved_table_ids, + unresolved_source_table_names, + }) +} + +async fn update_pending_flow_metadata( + context: &DdlContext, + flow_id: FlowId, + current_flow_info: &DeserializedValueWithBytes, + resolved_table_ids: Vec, + unresolved_source_table_names: Vec, + last_activation_error: Option, +) -> Result<()> { + let current_flow_info_value = current_flow_info.get_inner_ref(); + if current_flow_info_value.source_table_ids == resolved_table_ids + && current_flow_info_value.unresolved_source_table_names == unresolved_source_table_names + && current_flow_info_value.last_activation_error == last_activation_error + { + return Ok(()); + } + + let mut new_flow_info = current_flow_info.get_inner_ref().clone(); + new_flow_info.source_table_ids = resolved_table_ids; + new_flow_info.unresolved_source_table_names = unresolved_source_table_names; + new_flow_info.last_activation_error = last_activation_error; + new_flow_info.updated_time = chrono::Utc::now(); + context + .flow_metadata_manager + .update_flow_metadata(flow_id, current_flow_info, &new_flow_info, vec![]) + .await +} + +fn build_create_request( + flow_id: FlowId, + flow_info: &FlowInfoValue, + resolved_table_ids: &[TableId], +) -> CreateRequest { + let mut flow_options = flow_info.options.clone(); + flow_options.remove(DEFER_ON_MISSING_SOURCE_KEY); + flow_options.insert( + FlowType::FLOW_TYPE_KEY.to_string(), + get_flow_type(flow_info).to_string(), + ); + CreateRequest { + flow_id: Some(api::v1::FlowId { id: flow_id }), + source_table_ids: resolved_table_ids + .iter() + .map(|table_id| api::v1::TableId { id: *table_id }) + .collect_vec(), + sink_table_name: Some(flow_info.sink_table_name.clone().into()), + create_if_not_exists: false, + or_replace: true, + expire_after: flow_info.expire_after.map(|value| ExpireAfter { value }), + eval_interval: flow_info + .eval_interval_secs + .map(|seconds| api::v1::EvalInterval { seconds }), + comment: flow_info.comment.clone(), + sql: flow_info.raw_sql.clone(), + flow_options, + } +} + +async fn create_flow_on_peers( + context: &DdlContext, + peers: &[Peer], + request: CreateRequest, + flow_info: &FlowInfoValue, +) -> Result<()> { + let query_context = flow_info.query_context.clone().unwrap_or_default(); + let mut create_flow_tasks = Vec::with_capacity(peers.len()); + for peer in peers { + let requester = context.node_manager.flownode(peer).await; + let request = FlowRequest { + header: Some(FlowRequestHeader { + tracing_context: TracingContext::from_current_span().to_w3c(), + query_context: Some(query_context.clone().into()), + }), + body: Some(PbFlowRequest::Create(request.clone())), + }; + create_flow_tasks.push(async move { + requester + .handle(request) + .await + .map_err(add_peer_context_if_needed(peer.clone())) + }); + } + join_all(create_flow_tasks) + .await + .into_iter() + .collect::>>()?; + Ok(()) +} + +fn build_active_flow_info( + current_flow_info: &FlowInfoValue, + peers: &[Peer], + resolved_table_ids: &[TableId], +) -> (FlowInfoValue, Vec<(FlowPartitionId, FlowRouteValue)>) { + let flow_type = get_flow_type(current_flow_info); + let flownode_ids = peers + .iter() + .enumerate() + .map(|(idx, peer)| (idx as u32, peer.id)) + .collect::>(); + let flow_routes = peers + .iter() + .enumerate() + .map(|(idx, peer)| (idx as u32, FlowRouteValue { peer: peer.clone() })) + .collect::>(); + + let mut new_flow_info = current_flow_info.clone(); + new_flow_info.source_table_ids = resolved_table_ids.to_vec(); + new_flow_info.unresolved_source_table_names = vec![]; + new_flow_info.flownode_ids = flownode_ids; + new_flow_info.status = FlowStatus::Active; + new_flow_info.last_activation_error = None; + new_flow_info + .options + .insert(FlowType::FLOW_TYPE_KEY.to_string(), flow_type.to_string()); + new_flow_info.updated_time = chrono::Utc::now(); + + (new_flow_info, flow_routes) +} + +fn get_flow_type(flow_info: &FlowInfoValue) -> FlowType { + match flow_info + .options() + .get(FlowType::FLOW_TYPE_KEY) + .map(String::as_str) + { + Some(FlowType::STREAMING) => FlowType::Streaming, + _ => FlowType::Batching, + } +} + +async fn invalidate_flow_cache( + context: &DdlContext, + flow_id: FlowId, + resolved_table_ids: &[TableId], + peers: &[Peer], +) -> Result<()> { + let ctx = Context { + subject: Some("Invalidate flow cache by activating pending flow".to_string()), + }; + let flow_part2peers = peers + .iter() + .enumerate() + .map(|(idx, peer)| (idx as u32, peer.clone())) + .collect(); + context + .cache_invalidator + .invalidate( + &ctx, + &[ + CacheIdent::CreateFlow(CreateFlow { + flow_id, + source_table_ids: resolved_table_ids.to_vec(), + partition_to_peer_mapping: flow_part2peers, + }), + CacheIdent::FlowId(flow_id), + ], + ) + .await +} diff --git a/src/common/meta/src/ddl/create_flow.rs b/src/common/meta/src/ddl/create_flow.rs index 7120e5042597..1265d09a5029 100644 --- a/src/common/meta/src/ddl/create_flow.rs +++ b/src/common/meta/src/ddl/create_flow.rs @@ -14,12 +14,12 @@ mod metadata; -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use std::fmt; use api::v1::ExpireAfter; use api::v1::flow::flow_request::Body as PbFlowRequest; -use api::v1::flow::{CreateRequest, FlowRequest, FlowRequestHeader}; +use api::v1::flow::{CreateRequest, DropRequest, FlowRequest, FlowRequestHeader, flow_request}; use async_trait::async_trait; use common_catalog::format_full_flow_name; use common_procedure::error::{FromJsonSnafu, ToJsonSnafu}; @@ -34,13 +34,14 @@ use serde::{Deserialize, Serialize}; use snafu::{ResultExt, ensure}; use strum::AsRefStr; use table::metadata::TableId; +use table::table_name::TableName; use crate::cache_invalidator::Context; use crate::ddl::DdlContext; use crate::ddl::utils::{add_peer_context_if_needed, map_to_procedure_error}; use crate::error::{self, Result, UnexpectedSnafu}; use crate::instruction::{CacheIdent, CreateFlow, DropFlow}; -use crate::key::flow::flow_info::FlowInfoValue; +use crate::key::flow::flow_info::{FlowInfoValue, FlowStatus}; use crate::key::flow::flow_route::FlowRouteValue; use crate::key::table_name::TableNameKey; use crate::key::{DeserializedValueWithBytes, FlowId, FlowPartitionId}; @@ -67,11 +68,14 @@ impl CreateFlowProcedure { flow_id: None, peers: vec![], source_table_ids: vec![], + unresolved_source_table_names: vec![], flow_context: query_context.into(), // Convert to FlowQueryContext state: CreateFlowState::Prepare, prev_flow_info_value: None, + prev_peers: vec![], did_replace: false, flow_type: None, + last_activation_error: None, }, } } @@ -129,7 +133,8 @@ impl CreateFlowProcedure { .map(|(_, value)| value.peer) .collect::>(); self.data.flow_id = Some(flow_id); - self.data.peers = peers; + self.data.peers.clone_from(&peers); + self.data.prev_peers = peers; info!("Replacing flow, flow_id: {}", flow_id); let flow_info_value = self @@ -167,6 +172,20 @@ impl CreateFlowProcedure { } self.collect_source_tables().await?; + ensure!( + self.data.unresolved_source_table_names.is_empty() + || defer_on_missing_source(&self.data.task)?, + error::UnsupportedSnafu { + operation: format!( + "Create flow with missing source tables requires WITH ('{DEFER_ON_MISSING_SOURCE_KEY}'='true'): {}", + self.data + .unresolved_source_table_names + .iter() + .map(ToString::to_string) + .join(", ") + ) + } + ); // Validate that source and sink tables are not the same let sink_table_name = &self.data.task.sink_table_name; @@ -189,10 +208,20 @@ impl CreateFlowProcedure { if self.data.flow_id.is_none() { self.allocate_flow_id().await?; } - self.data.state = CreateFlowState::CreateFlows; - // determine flow type self.data.flow_type = Some(get_flow_type_from_options(&self.data.task)?); + self.data.state = if self.data.is_pending() { + self.data.peers.clear(); + CreateFlowState::CreateMetadata + } else { + if self.data.peers.is_empty() { + // Replacing a pending flow means the flow id already exists, but there + // are no flow routes yet. Allocate peers for the newly-active flow. + self.data.peers = self.context.flow_metadata_allocator.alloc_peers(1).await?; + } + CreateFlowState::CreateFlows + }; + Ok(Status::executing(true)) } @@ -301,8 +330,45 @@ impl CreateFlowProcedure { .invalidate(&ctx, &caches) .await?; + if did_replace { + self.on_flownode_drop_previous_flows().await?; + } + Ok(Status::done_with_output(flow_id)) } + + async fn on_flownode_drop_previous_flows(&self) -> Result<()> { + let Some(prev_flow_info) = self.data.prev_flow_info_value.as_ref() else { + return Ok(()); + }; + if prev_flow_info.flownode_ids().is_empty() { + return Ok(()); + } + + let flow_id = self.data.flow_id.unwrap(); + let mut drop_flow_tasks = Vec::with_capacity(self.data.prev_peers.len()); + for peer in &self.data.prev_peers { + let requester = self.context.node_manager.flownode(peer).await; + let request = FlowRequest { + body: Some(flow_request::Body::Drop(DropRequest { + flow_id: Some(api::v1::FlowId { id: flow_id }), + })), + ..Default::default() + }; + drop_flow_tasks.push(async move { + requester + .handle(request) + .await + .map_err(add_peer_context_if_needed(peer.clone())) + }); + } + join_all(drop_flow_tasks) + .await + .into_iter() + .collect::>>()?; + + Ok(()) + } } #[async_trait] @@ -365,6 +431,40 @@ pub fn get_flow_type_from_options(flow_task: &CreateFlowTask) -> Result Result { + flow_task + .flow_options + .get(DEFER_ON_MISSING_SOURCE_KEY) + .map(|value| { + value + .trim() + .to_ascii_lowercase() + .parse::() + .map_err(|_| { + error::UnexpectedSnafu { + err_msg: format!( + "Invalid flow option '{DEFER_ON_MISSING_SOURCE_KEY}': {value}" + ), + } + .build() + }) + }) + .transpose() + .map(|value| value.unwrap_or(false)) +} + +fn user_runtime_flow_options(options: &HashMap) -> HashMap { + let mut options = options.clone(); + options.remove(DEFER_ON_MISSING_SOURCE_KEY); + options +} + +fn metadata_flow_options(options: &HashMap) -> HashMap { + options.clone() +} + /// The state of [CreateFlowProcedure]. #[derive(Debug, Clone, Serialize, Deserialize, AsRefStr, PartialEq)] pub enum CreateFlowState { @@ -411,17 +511,33 @@ pub struct CreateFlowData { pub(crate) flow_id: Option, pub(crate) peers: Vec, pub(crate) source_table_ids: Vec, + #[serde(default)] + pub(crate) unresolved_source_table_names: Vec, /// Use alias for backward compatibility with QueryContext serialized data #[serde(alias = "query_context")] pub(crate) flow_context: FlowQueryContext, /// For verify if prev value is consistent when need to update flow metadata. /// only set when `or_replace` is true. pub(crate) prev_flow_info_value: Option>, + #[serde(default)] + pub(crate) prev_peers: Vec, /// Only set to true when replace actually happened. /// This is used to determine whether to invalidate the cache. #[serde(default)] pub(crate) did_replace: bool, pub(crate) flow_type: Option, + #[serde(default)] + pub(crate) last_activation_error: Option, +} + +impl CreateFlowData { + pub(crate) fn is_pending(&self) -> bool { + !self.unresolved_source_table_names.is_empty() + } + + pub(crate) fn is_active(&self) -> bool { + !self.is_pending() + } } impl From<&CreateFlowData> for CreateRequest { @@ -446,7 +562,7 @@ impl From<&CreateFlowData> for CreateRequest { .map(|seconds| api::v1::EvalInterval { seconds }), comment: value.task.comment.clone(), sql: value.task.sql.clone(), - flow_options: value.task.flow_options.clone(), + flow_options: user_runtime_flow_options(&value.task.flow_options), }; let flow_type = value.flow_type.unwrap_or_default().to_string(); @@ -466,9 +582,9 @@ impl From<&CreateFlowData> for (FlowInfoValue, Vec<(FlowPartitionId, FlowRouteVa eval_interval_secs: eval_interval, comment, sql, - flow_options: mut options, .. } = value.task.clone(); + let mut options = metadata_flow_options(&value.task.flow_options); let flownode_ids = value .peers @@ -495,6 +611,8 @@ impl From<&CreateFlowData> for (FlowInfoValue, Vec<(FlowPartitionId, FlowRouteVa let flow_info: FlowInfoValue = FlowInfoValue { source_table_ids: value.source_table_ids.clone(), + all_source_table_names: value.task.source_table_names.clone(), + unresolved_source_table_names: value.unresolved_source_table_names.clone(), sink_table_name, flownode_ids, catalog_name, @@ -506,6 +624,12 @@ impl From<&CreateFlowData> for (FlowInfoValue, Vec<(FlowPartitionId, FlowRouteVa eval_interval_secs: eval_interval, comment, options, + status: if value.is_active() { + FlowStatus::Active + } else { + FlowStatus::PendingSources + }, + last_activation_error: value.last_activation_error.clone(), created_time: create_time, updated_time: chrono::Utc::now(), }; diff --git a/src/common/meta/src/ddl/create_flow/metadata.rs b/src/common/meta/src/ddl/create_flow/metadata.rs index 27b85b794634..f97ecfdf4a86 100644 --- a/src/common/meta/src/ddl/create_flow/metadata.rs +++ b/src/common/meta/src/ddl/create_flow/metadata.rs @@ -12,10 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use snafu::OptionExt; - use crate::ddl::create_flow::CreateFlowProcedure; -use crate::error::{self, Result}; +use crate::error::Result; use crate::key::table_name::TableNameKey; impl CreateFlowProcedure { @@ -34,9 +32,8 @@ impl CreateFlowProcedure { Ok(()) } - /// Ensures all source tables exist and collects source table ids + /// Collects source table ids and keeps track of missing tables. pub(crate) async fn collect_source_tables(&mut self) -> Result<()> { - // Ensures all source tables exist. let keys = self .data .task @@ -52,22 +49,24 @@ impl CreateFlowProcedure { .batch_get(keys) .await?; - let source_table_ids = self + let mut resolved = Vec::with_capacity(self.data.task.source_table_names.len()); + let mut unresolved = Vec::new(); + + for (name, table_id) in self .data .task .source_table_names .iter() .zip(source_table_ids) - .map(|(name, table_id)| { - Ok(table_id - .with_context(|| error::TableNotFoundSnafu { - table_name: name.to_string(), - })? - .table_id()) - }) - .collect::>>()?; + { + match table_id { + Some(table_id) => resolved.push(table_id.table_id()), + None => unresolved.push(name.clone()), + } + } - self.data.source_table_ids = source_table_ids; + self.data.source_table_ids = resolved; + self.data.unresolved_source_table_names = unresolved; Ok(()) } } diff --git a/src/common/meta/src/ddl/flow_meta.rs b/src/common/meta/src/ddl/flow_meta.rs index 85c1f3e989cc..fd41690b260e 100644 --- a/src/common/meta/src/ddl/flow_meta.rs +++ b/src/common/meta/src/ddl/flow_meta.rs @@ -59,8 +59,13 @@ impl FlowMetadataAllocator { /// Allocates the [FlowId] and [Peer]s. pub async fn create(&self, partitions: usize) -> Result<(FlowId, Vec)> { let flow_id = self.allocate_flow_id().await?; - let peers = self.peer_allocator.alloc(partitions).await?; + let peers = self.alloc_peers(partitions).await?; Ok((flow_id, peers)) } + + pub async fn alloc_peers(&self, partitions: usize) -> Result> { + let peers = self.peer_allocator.alloc(partitions).await?; + Ok(peers) + } } diff --git a/src/common/meta/src/ddl/tests.rs b/src/common/meta/src/ddl/tests.rs index 0700259cf844..a09283205bf2 100644 --- a/src/common/meta/src/ddl/tests.rs +++ b/src/common/meta/src/ddl/tests.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod activate_flow; mod alter_logical_tables; mod alter_table; mod create_flow; diff --git a/src/common/meta/src/ddl/tests/activate_flow.rs b/src/common/meta/src/ddl/tests/activate_flow.rs new file mode 100644 index 000000000000..9502b5dcb020 --- /dev/null +++ b/src/common/meta/src/ddl/tests/activate_flow.rs @@ -0,0 +1,329 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::{Arc, Mutex}; + +use api::v1::flow::flow_request::Body as PbFlowRequest; +use api::v1::flow::{CreateRequest, FlowRequest, FlowResponse}; +use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; +use common_procedure_test::execute_procedure_until_done; +use common_time::TimeToLive; +use table::table_name::TableName; + +use crate::ddl::activate_flow::ActivatePendingFlowProcedure; +use crate::ddl::create_flow::{DEFER_ON_MISSING_SOURCE_KEY, FlowType}; +use crate::ddl::test_util::create_table::test_create_table_task; +use crate::ddl::tests::create_flow::create_test_pending_flow; +use crate::key::table_route::TableRouteValue; +use crate::test_util::{MockFlownodeHandler, MockFlownodeManager, new_ddl_context}; + +#[derive(Clone, Default)] +struct RecordingFlownodeHandler { + create_requests: Arc>>, +} + +#[async_trait::async_trait] +impl MockFlownodeHandler for RecordingFlownodeHandler { + async fn handle( + &self, + _peer: &crate::peer::Peer, + request: FlowRequest, + ) -> crate::error::Result { + if let Some(PbFlowRequest::Create(create_req)) = request.body { + self.create_requests.lock().unwrap().push(create_req); + } + + Ok(FlowResponse { + affected_rows: 0, + ..Default::default() + }) + } + + async fn handle_inserts( + &self, + _peer: &crate::peer::Peer, + _requests: api::v1::region::InsertRequests, + ) -> crate::error::Result { + unreachable!() + } +} + +#[tokio::test] +async fn test_activate_pending_flow() { + let source_table_names = vec![TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "activate_source_table", + )]; + let sink_table_name = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "activate_sink_table", + ); + let node_manager = Arc::new(MockFlownodeManager::new( + crate::ddl::test_util::flownode_handler::NaiveFlownodeHandler, + )); + let ddl_context = new_ddl_context(node_manager); + + let flow_id = create_test_pending_flow( + &ddl_context, + "activate_pending_flow", + source_table_names.clone(), + sink_table_name, + ) + .await; + + let pending_flow = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(flow_id) + .await + .unwrap() + .unwrap(); + assert!(pending_flow.is_pending()); + + let create_table_task = test_create_table_task("activate_source_table", 1024); + ddl_context + .table_metadata_manager + .create_table_metadata( + create_table_task.table_info.clone(), + TableRouteValue::physical(vec![]), + Default::default(), + ) + .await + .unwrap(); + + let mut procedure = ActivatePendingFlowProcedure::new( + flow_id, + DEFAULT_CATALOG_NAME.to_string(), + ddl_context.clone(), + ); + let output = execute_procedure_until_done(&mut procedure).await.unwrap(); + let activated_flow_id = output.downcast_ref::().unwrap(); + assert_eq!(*activated_flow_id, flow_id); + + let activated_flow = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(flow_id) + .await + .unwrap() + .unwrap(); + assert!(activated_flow.is_active()); + assert_eq!(activated_flow.unresolved_source_table_names().len(), 0); + assert_eq!(activated_flow.source_table_ids(), &[1024]); + assert_eq!(activated_flow.last_activation_error(), &None); + assert!(!activated_flow.flownode_ids().is_empty()); +} + +#[tokio::test] +async fn test_activate_pending_batching_flow_with_instant_ttl_source() { + let source_table_names = vec![TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "instant_ttl_source_table", + )]; + let sink_table_name = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "instant_ttl_sink_table", + ); + let node_manager = Arc::new(MockFlownodeManager::new( + crate::ddl::test_util::flownode_handler::NaiveFlownodeHandler, + )); + let ddl_context = new_ddl_context(node_manager); + + let flow_id = create_test_pending_flow( + &ddl_context, + "instant_ttl_pending_flow", + source_table_names.clone(), + sink_table_name, + ) + .await; + + let mut create_table_task = test_create_table_task("instant_ttl_source_table", 1025); + create_table_task.table_info.meta.options.ttl = Some(TimeToLive::Instant); + ddl_context + .table_metadata_manager + .create_table_metadata( + create_table_task.table_info.clone(), + TableRouteValue::physical(vec![]), + Default::default(), + ) + .await + .unwrap(); + + let mut procedure = ActivatePendingFlowProcedure::new( + flow_id, + DEFAULT_CATALOG_NAME.to_string(), + ddl_context.clone(), + ); + let output = execute_procedure_until_done(&mut procedure).await.unwrap(); + let activated_flow_id = output.downcast_ref::().unwrap(); + assert_eq!(*activated_flow_id, flow_id); + + let activated_flow = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(flow_id) + .await + .unwrap() + .unwrap(); + assert!(activated_flow.is_active()); + assert_eq!(activated_flow.unresolved_source_table_names().len(), 0); + assert_eq!(activated_flow.source_table_ids(), &[1025]); + assert_eq!(activated_flow.last_activation_error(), &None); +} + +#[tokio::test] +async fn test_activate_pending_flow_uses_replace_semantics() { + let source_table_names = vec![TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "activate_replace_source_table", + )]; + let sink_table_name = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "activate_replace_sink_table", + ); + + let handler = RecordingFlownodeHandler::default(); + let node_manager = Arc::new(MockFlownodeManager::new(handler.clone())); + let ddl_context = new_ddl_context(node_manager); + + let flow_id = create_test_pending_flow( + &ddl_context, + "activate_pending_flow_replace_semantics", + source_table_names, + sink_table_name, + ) + .await; + + let create_table_task = test_create_table_task("activate_replace_source_table", 1027); + ddl_context + .table_metadata_manager + .create_table_metadata( + create_table_task.table_info.clone(), + TableRouteValue::physical(vec![]), + Default::default(), + ) + .await + .unwrap(); + + let mut procedure = + ActivatePendingFlowProcedure::new(flow_id, DEFAULT_CATALOG_NAME.to_string(), ddl_context); + let output = execute_procedure_until_done(&mut procedure).await.unwrap(); + let activated_flow_id = output.downcast_ref::().unwrap(); + assert_eq!(*activated_flow_id, flow_id); + + let create_requests = handler.create_requests.lock().unwrap(); + assert!(!create_requests.is_empty()); + for req in create_requests.iter() { + assert!(!req.create_if_not_exists); + assert!(req.or_replace); + } +} + +#[tokio::test] +async fn test_activate_pending_flow_preserves_streaming_type() { + let source_table_names = vec![TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "activate_streaming_source_table", + )]; + let sink_table_name = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "activate_streaming_sink_table", + ); + + let handler = RecordingFlownodeHandler::default(); + let node_manager = Arc::new(MockFlownodeManager::new(handler.clone())); + let ddl_context = new_ddl_context(node_manager); + + let flow_id = create_test_pending_flow( + &ddl_context, + "activate_streaming_pending_flow", + source_table_names, + sink_table_name, + ) + .await; + + let pending_flow_info = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get_raw(flow_id) + .await + .unwrap() + .unwrap(); + let mut updated_flow_info = pending_flow_info.get_inner_ref().clone(); + updated_flow_info.options.insert( + FlowType::FLOW_TYPE_KEY.to_string(), + FlowType::Streaming.to_string(), + ); + ddl_context + .flow_metadata_manager + .update_flow_metadata(flow_id, &pending_flow_info, &updated_flow_info, vec![]) + .await + .unwrap(); + + let create_table_task = test_create_table_task("activate_streaming_source_table", 1028); + ddl_context + .table_metadata_manager + .create_table_metadata( + create_table_task.table_info.clone(), + TableRouteValue::physical(vec![]), + Default::default(), + ) + .await + .unwrap(); + + let mut procedure = ActivatePendingFlowProcedure::new( + flow_id, + DEFAULT_CATALOG_NAME.to_string(), + ddl_context.clone(), + ); + let output = execute_procedure_until_done(&mut procedure).await.unwrap(); + let activated_flow_id = output.downcast_ref::().unwrap(); + assert_eq!(*activated_flow_id, flow_id); + + let activated_flow = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(flow_id) + .await + .unwrap() + .unwrap(); + assert!(activated_flow.is_active()); + assert_eq!( + activated_flow + .options() + .get(FlowType::FLOW_TYPE_KEY) + .map(String::as_str), + Some(FlowType::STREAMING) + ); + + let create_requests = handler.create_requests.lock().unwrap(); + assert!(!create_requests.is_empty()); + for req in create_requests.iter() { + assert!(!req.flow_options.contains_key(DEFER_ON_MISSING_SOURCE_KEY)); + assert_eq!( + req.flow_options + .get(FlowType::FLOW_TYPE_KEY) + .map(String::as_str), + Some(FlowType::STREAMING) + ); + } +} diff --git a/src/common/meta/src/ddl/tests/create_flow.rs b/src/common/meta/src/ddl/tests/create_flow.rs index 344fc05024e6..fd704c6bfaf0 100644 --- a/src/common/meta/src/ddl/tests/create_flow.rs +++ b/src/common/meta/src/ddl/tests/create_flow.rs @@ -14,14 +14,20 @@ use std::assert_matches; use std::collections::HashMap; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; +use api::v1::flow::flow_request::Body as PbFlowRequest; +use api::v1::flow::{CreateRequest, DropRequest, FlowRequest, FlowResponse}; use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; +use common_procedure::Status; use common_procedure_test::execute_procedure_until_done; use table::table_name::TableName; use crate::ddl::DdlContext; -use crate::ddl::create_flow::{CreateFlowData, CreateFlowProcedure, CreateFlowState, FlowType}; +use crate::ddl::create_flow::{ + CreateFlowData, CreateFlowProcedure, CreateFlowState, DEFER_ON_MISSING_SOURCE_KEY, FlowType, + defer_on_missing_source, +}; use crate::ddl::test_util::create_table::test_create_table_task; use crate::ddl::test_util::flownode_handler::NaiveFlownodeHandler; use crate::error; @@ -30,6 +36,37 @@ use crate::key::table_route::TableRouteValue; use crate::rpc::ddl::{CreateFlowTask, FlowQueryContext, QueryContext}; use crate::test_util::{MockFlownodeManager, new_ddl_context}; +#[derive(Clone, Default)] +struct RecordingDropFlownodeHandler { + drop_requests: Arc>>, +} + +#[async_trait::async_trait] +impl crate::test_util::MockFlownodeHandler for RecordingDropFlownodeHandler { + async fn handle( + &self, + _peer: &crate::peer::Peer, + request: FlowRequest, + ) -> crate::error::Result { + if let Some(PbFlowRequest::Drop(drop_req)) = request.body { + self.drop_requests.lock().unwrap().push(drop_req); + } + + Ok(FlowResponse { + affected_rows: 0, + ..Default::default() + }) + } + + async fn handle_inserts( + &self, + _peer: &crate::peer::Peer, + _requests: api::v1::region::InsertRequests, + ) -> crate::error::Result { + unreachable!() + } +} + fn test_query_context() -> QueryContext { QueryContext { current_catalog: DEFAULT_CATALOG_NAME.to_string(), @@ -63,6 +100,11 @@ pub(crate) fn test_create_flow_task( } } +fn enable_defer_on_missing_source(task: &mut CreateFlowTask) { + task.flow_options + .insert(DEFER_ON_MISSING_SOURCE_KEY.to_string(), "true".to_string()); +} + #[tokio::test] async fn test_create_flow_source_table_not_found() { let source_table_names = vec![TableName::new( @@ -78,7 +120,145 @@ async fn test_create_flow_source_table_not_found() { let query_ctx = test_query_context(); let mut procedure = CreateFlowProcedure::new(task, query_ctx, ddl_context); let err = procedure.on_prepare().await.unwrap_err(); - assert_matches!(err, error::Error::TableNotFound { .. }); + assert_matches!(err, error::Error::Unsupported { .. }); + assert!( + err.to_string() + .contains("requires WITH ('defer_on_missing_source'='true')") + ); +} + +#[tokio::test] +async fn test_create_pending_flow_source_table_not_found_with_defer() { + let source_table_names = vec![TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "my_table", + )]; + let sink_table_name = + TableName::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "my_sink_table"); + let mut task = test_create_flow_task("my_flow", source_table_names, sink_table_name, false); + enable_defer_on_missing_source(&mut task); + let node_manager = Arc::new(MockFlownodeManager::new(NaiveFlownodeHandler)); + let ddl_context = new_ddl_context(node_manager); + let query_ctx = test_query_context(); + let mut procedure = CreateFlowProcedure::new(task, query_ctx, ddl_context.clone()); + let status = procedure.on_prepare().await.unwrap(); + assert_matches!(status, Status::Executing { persist: true, .. }); + assert_eq!(procedure.data.unresolved_source_table_names.len(), 1); + assert_eq!(procedure.data.source_table_ids, Vec::::new()); + + let output = execute_procedure_until_done(&mut procedure).await.unwrap(); + let flow_id = *output.downcast_ref::().unwrap(); + let flow_info = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(flow_id) + .await + .unwrap() + .unwrap(); + assert_eq!(flow_info.source_table_ids(), Vec::::new()); + assert_eq!( + flow_info + .options() + .get(DEFER_ON_MISSING_SOURCE_KEY) + .map(String::as_str), + Some("true") + ); +} + +#[test] +fn test_defer_on_missing_source_defaults_false() { + let task = test_create_flow_task( + "my_flow", + vec![], + TableName::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "my_sink_table"), + false, + ); + + assert!(!defer_on_missing_source(&task).unwrap()); +} + +#[test] +fn test_defer_on_missing_source_true() { + let mut task = test_create_flow_task( + "my_flow", + vec![], + TableName::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "my_sink_table"), + false, + ); + task.flow_options + .insert(DEFER_ON_MISSING_SOURCE_KEY.to_string(), "true".to_string()); + + assert!(defer_on_missing_source(&task).unwrap()); +} + +#[test] +fn test_defer_on_missing_source_invalid_value() { + let mut task = test_create_flow_task( + "my_flow", + vec![], + TableName::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "my_sink_table"), + false, + ); + task.flow_options.insert( + DEFER_ON_MISSING_SOURCE_KEY.to_string(), + "invalid".to_string(), + ); + + let err = defer_on_missing_source(&task).unwrap_err(); + assert!( + err.to_string() + .contains("Invalid flow option 'defer_on_missing_source': invalid") + ); +} + +#[test] +fn test_create_request_strips_defer_on_missing_source_runtime_option() { + let mut task = test_create_flow_task( + "my_flow", + vec![], + TableName::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "my_sink_table"), + false, + ); + enable_defer_on_missing_source(&mut task); + + let data = CreateFlowData { + state: CreateFlowState::CreateFlows, + task, + flow_id: Some(1024), + peers: vec![], + source_table_ids: vec![], + unresolved_source_table_names: vec![], + flow_context: FlowQueryContext { + catalog: DEFAULT_CATALOG_NAME.to_string(), + schema: DEFAULT_SCHEMA_NAME.to_string(), + timezone: "UTC".to_string(), + extensions: HashMap::new(), + channel: 0, + snapshot_seqs: HashMap::new(), + sst_min_sequences: HashMap::new(), + }, + prev_flow_info_value: None, + prev_peers: vec![], + did_replace: false, + flow_type: Some(FlowType::Batching), + last_activation_error: None, + }; + + let request: CreateRequest = (&data).into(); + + assert!( + !request + .flow_options + .contains_key(DEFER_ON_MISSING_SOURCE_KEY) + ); + assert_eq!( + request + .flow_options + .get(FlowType::FLOW_TYPE_KEY) + .map(String::as_str), + Some(FlowType::BATCHING) + ); } pub(crate) async fn create_test_flow( @@ -101,6 +281,27 @@ pub(crate) async fn create_test_flow( *flow_id } +pub(crate) async fn create_test_pending_flow( + ddl_context: &DdlContext, + flow_name: &str, + source_table_names: Vec, + sink_table_name: TableName, +) -> FlowId { + let mut task = test_create_flow_task( + flow_name, + source_table_names.clone(), + sink_table_name.clone(), + false, + ); + enable_defer_on_missing_source(&mut task); + let query_ctx = test_query_context(); + let mut procedure = CreateFlowProcedure::new(task, query_ctx, ddl_context.clone()); + let output = execute_procedure_until_done(&mut procedure).await.unwrap(); + let flow_id = output.downcast_ref::().unwrap(); + + *flow_id +} + #[tokio::test] async fn test_create_flow() { let table_id = 1024; @@ -154,6 +355,159 @@ async fn test_create_flow() { assert_matches!(err, error::Error::FlowAlreadyExists { .. }); } +#[tokio::test] +async fn test_replace_pending_flow_activates_with_allocated_peers() { + let source_table_name = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "replace_pending_source_table", + ); + let sink_table_name = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "replace_pending_sink_table", + ); + let node_manager = Arc::new(MockFlownodeManager::new(NaiveFlownodeHandler)); + let ddl_context = new_ddl_context(node_manager); + + let pending_flow_id = create_test_pending_flow( + &ddl_context, + "replace_pending_flow", + vec![source_table_name.clone()], + sink_table_name.clone(), + ) + .await; + + let pending_flow = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(pending_flow_id) + .await + .unwrap() + .unwrap(); + assert!(pending_flow.is_pending()); + assert!(pending_flow.flownode_ids().is_empty()); + + let create_table_task = test_create_table_task("replace_pending_source_table", 1026); + ddl_context + .table_metadata_manager + .create_table_metadata( + create_table_task.table_info.clone(), + TableRouteValue::physical(vec![]), + HashMap::new(), + ) + .await + .unwrap(); + + let mut replace_task = test_create_flow_task( + "replace_pending_flow", + vec![source_table_name], + sink_table_name, + false, + ); + replace_task.or_replace = true; + let query_ctx = test_query_context(); + let mut procedure = CreateFlowProcedure::new(replace_task, query_ctx, ddl_context.clone()); + let output = execute_procedure_until_done(&mut procedure).await.unwrap(); + let flow_id = *output.downcast_ref::().unwrap(); + assert_eq!(flow_id, pending_flow_id); + + let replaced_flow = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(flow_id) + .await + .unwrap() + .unwrap(); + assert!(replaced_flow.is_active()); + assert_eq!(replaced_flow.source_table_ids(), &[1026]); + assert!(!replaced_flow.flownode_ids().is_empty()); + + let routes = ddl_context + .flow_metadata_manager + .flow_route_manager() + .routes(flow_id) + .await + .unwrap(); + assert!(!routes.is_empty()); +} + +#[tokio::test] +async fn test_replace_active_flow_to_pending_drops_old_flows() { + let existing_source_table = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "replace_active_source_table", + ); + let missing_source_table = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "replace_missing_source_table", + ); + let sink_table_name = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "replace_active_sink_table", + ); + + let handler = RecordingDropFlownodeHandler::default(); + let node_manager = Arc::new(MockFlownodeManager::new(handler.clone())); + let ddl_context = new_ddl_context(node_manager); + + let create_table_task = test_create_table_task("replace_active_source_table", 2026); + ddl_context + .table_metadata_manager + .create_table_metadata( + create_table_task.table_info.clone(), + TableRouteValue::physical(vec![]), + HashMap::new(), + ) + .await + .unwrap(); + + let flow_id = create_test_flow( + &ddl_context, + "replace_active_flow_to_pending", + vec![existing_source_table], + sink_table_name.clone(), + ) + .await; + + let mut replace_task = test_create_flow_task( + "replace_active_flow_to_pending", + vec![missing_source_table], + sink_table_name, + false, + ); + enable_defer_on_missing_source(&mut replace_task); + replace_task.or_replace = true; + let query_ctx = test_query_context(); + let mut procedure = CreateFlowProcedure::new(replace_task, query_ctx, ddl_context.clone()); + let output = execute_procedure_until_done(&mut procedure).await.unwrap(); + let replaced_flow_id = *output.downcast_ref::().unwrap(); + assert_eq!(replaced_flow_id, flow_id); + + let replaced_flow = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(flow_id) + .await + .unwrap() + .unwrap(); + assert!(replaced_flow.is_pending()); + assert_eq!( + replaced_flow + .options() + .get(DEFER_ON_MISSING_SOURCE_KEY) + .map(String::as_str), + Some("true") + ); + + let drop_requests = handler.drop_requests.lock().unwrap(); + assert!(!drop_requests.is_empty()); + assert_eq!(drop_requests[0].flow_id.as_ref().unwrap().id, flow_id); +} + #[tokio::test] async fn test_create_flow_same_source_and_sink_table() { let table_id = 1024; @@ -228,6 +582,7 @@ fn test_create_flow_data_serialization_backward_compatibility() { "flow_id": null, "peers": [], "source_table_ids": [], + "unresolved_source_table_names": [], "query_context": { "current_catalog": "old_catalog", "current_schema": "old_schema", @@ -265,10 +620,13 @@ fn test_create_flow_data_new_format_serialization() { flow_id: None, peers: vec![], source_table_ids: vec![], + unresolved_source_table_names: vec![], flow_context, prev_flow_info_value: None, + prev_peers: vec![], did_replace: false, flow_type: None, + last_activation_error: None, }; let serialized = serde_json::to_string(&data).unwrap(); @@ -327,10 +685,13 @@ fn test_flow_info_conversion_with_flow_context() { flow_id: Some(123), peers: vec![], source_table_ids: vec![456, 789], + unresolved_source_table_names: vec![], flow_context, prev_flow_info_value: None, + prev_peers: vec![], did_replace: false, flow_type: Some(FlowType::Batching), + last_activation_error: None, }; let (flow_info, _routes) = (&data).into(); diff --git a/src/common/meta/src/ddl_manager.rs b/src/common/meta/src/ddl_manager.rs index d0619ca74fbd..2f5dc928fc5c 100644 --- a/src/common/meta/src/ddl_manager.rs +++ b/src/common/meta/src/ddl_manager.rs @@ -29,6 +29,7 @@ use snafu::{OptionExt, ResultExt, ensure}; use store_api::storage::TableId; use table::table_name::TableName; +use crate::ddl::activate_flow::ActivatePendingFlowProcedure; use crate::ddl::alter_database::AlterDatabaseProcedure; use crate::ddl::alter_logical_tables::AlterLogicalTablesProcedure; use crate::ddl::alter_table::AlterTableProcedure; @@ -232,6 +233,7 @@ impl DdlManager { CreateLogicalTablesProcedure, CreateViewProcedure, CreateFlowProcedure, + ActivatePendingFlowProcedure, AlterTableProcedure, AlterLogicalTablesProcedure, AlterDatabaseProcedure, @@ -488,6 +490,20 @@ impl DdlManager { self.execute_procedure_and_wait(procedure_with_id).await } + /// Submits and executes a pending-flow activation task. + #[tracing::instrument(skip_all)] + pub async fn submit_activate_pending_flow_task( + &self, + flow_id: crate::key::FlowId, + catalog_name: String, + ) -> Result<(ProcedureId, Option)> { + let context = self.create_context(); + let procedure = ActivatePendingFlowProcedure::new(flow_id, catalog_name, context); + let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure)); + + self.execute_procedure_and_wait(procedure_with_id).await + } + /// Submits and executes a drop flow task. #[tracing::instrument(skip_all)] pub async fn submit_drop_flow_task( diff --git a/src/common/meta/src/key/flow.rs b/src/common/meta/src/key/flow.rs index d581b926853d..e1223fd67f38 100644 --- a/src/common/meta/src/key/flow.rs +++ b/src/common/meta/src/key/flow.rs @@ -459,6 +459,7 @@ mod tests { use super::*; use crate::FlownodeId; + use crate::key::flow::flow_info::FlowStatus; use crate::key::flow::table_flow::TableFlowKey; use crate::key::node_address::{NodeAddressKey, NodeAddressValue}; use crate::key::{FlowPartitionId, MetadataValue}; @@ -522,6 +523,8 @@ mod tests { query_context: None, flow_name: flow_name.to_string(), source_table_ids, + all_source_table_names: vec![], + unresolved_source_table_names: vec![], sink_table_name, flownode_ids, raw_sql: "raw".to_string(), @@ -529,6 +532,8 @@ mod tests { eval_interval_secs: None, comment: "hi".to_string(), options: Default::default(), + status: FlowStatus::Active, + last_activation_error: None, created_time: chrono::Utc::now(), updated_time: chrono::Utc::now(), } @@ -774,6 +779,8 @@ mod tests { query_context: None, flow_name: "flow".to_string(), source_table_ids: vec![1024, 1025, 1026], + all_source_table_names: vec![], + unresolved_source_table_names: vec![], sink_table_name: another_sink_table_name, flownode_ids: [(0, 1u64)].into(), raw_sql: "raw".to_string(), @@ -781,6 +788,8 @@ mod tests { eval_interval_secs: None, comment: "hi".to_string(), options: Default::default(), + status: FlowStatus::Active, + last_activation_error: None, created_time: chrono::Utc::now(), updated_time: chrono::Utc::now(), }; @@ -1151,6 +1160,8 @@ mod tests { query_context: None, flow_name: "flow".to_string(), source_table_ids: vec![1024, 1025, 1026], + all_source_table_names: vec![], + unresolved_source_table_names: vec![], sink_table_name: another_sink_table_name, flownode_ids: [(0, 1u64)].into(), raw_sql: "raw".to_string(), @@ -1158,6 +1169,8 @@ mod tests { eval_interval_secs: None, comment: "hi".to_string(), options: Default::default(), + status: FlowStatus::Active, + last_activation_error: None, created_time: chrono::Utc::now(), updated_time: chrono::Utc::now(), }; diff --git a/src/common/meta/src/key/flow/flow_info.rs b/src/common/meta/src/key/flow/flow_info.rs index d501822c3c9e..0957d5681576 100644 --- a/src/common/meta/src/key/flow/flow_info.rs +++ b/src/common/meta/src/key/flow/flow_info.rs @@ -16,6 +16,8 @@ use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; use chrono::{DateTime, Utc}; +use futures::TryStreamExt; +use futures::stream::BoxStream; use lazy_static::lazy_static; use regex::Regex; use serde::{Deserialize, Serialize}; @@ -27,12 +29,24 @@ use crate::FlownodeId; use crate::error::{self, Result}; use crate::key::flow::FlowScoped; use crate::key::txn_helper::TxnOpGetResponseSet; -use crate::key::{DeserializedValueWithBytes, FlowId, FlowPartitionId, MetadataKey, MetadataValue}; +use crate::key::{ + BytesAdapter, DeserializedValueWithBytes, FlowId, FlowPartitionId, MetadataKey, MetadataValue, +}; use crate::kv_backend::KvBackendRef; use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp}; +use crate::range_stream::{DEFAULT_PAGE_SIZE, PaginationStream}; +use crate::rpc::KeyValue; +use crate::rpc::store::RangeRequest; pub const FLOW_INFO_KEY_PREFIX: &str = "info"; +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq, Default)] +pub enum FlowStatus { + PendingSources, + #[default] + Active, +} + lazy_static! { static ref FLOW_INFO_KEY_PATTERN: Regex = Regex::new(&format!("^{FLOW_INFO_KEY_PREFIX}/([0-9]+)$")).unwrap(); @@ -114,7 +128,12 @@ impl<'a> MetadataKey<'a, FlowInfoKeyInner> for FlowInfoKeyInner { #[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] pub struct FlowInfoValue { /// The source tables used by the flow. + #[serde(default)] pub source_table_ids: Vec, + #[serde(default)] + pub all_source_table_names: Vec, + #[serde(default)] + pub unresolved_source_table_names: Vec, /// The sink table used by the flow. pub sink_table_name: TableName, /// Which flow nodes this flow is running on. @@ -145,6 +164,10 @@ pub struct FlowInfoValue { pub comment: String, /// The options. pub options: HashMap, + #[serde(default)] + pub status: FlowStatus, + #[serde(default)] + pub last_activation_error: Option, /// The created time #[serde(default)] pub created_time: DateTime, @@ -154,6 +177,14 @@ pub struct FlowInfoValue { } impl FlowInfoValue { + pub fn is_pending(&self) -> bool { + self.status == FlowStatus::PendingSources + } + + pub fn is_active(&self) -> bool { + self.status == FlowStatus::Active + } + /// Returns the `flownode_id`. pub fn flownode_ids(&self) -> &BTreeMap { &self.flownode_ids @@ -173,6 +204,14 @@ impl FlowInfoValue { &self.source_table_ids } + pub fn all_source_table_names(&self) -> &[TableName] { + &self.all_source_table_names + } + + pub fn unresolved_source_table_names(&self) -> &[TableName] { + &self.unresolved_source_table_names + } + pub fn catalog_name(&self) -> &String { &self.catalog_name } @@ -209,6 +248,14 @@ impl FlowInfoValue { &self.options } + pub fn status(&self) -> &FlowStatus { + &self.status + } + + pub fn last_activation_error(&self) -> &Option { + &self.last_activation_error + } + pub fn created_time(&self) -> &DateTime { &self.created_time } @@ -225,6 +272,12 @@ pub struct FlowInfoManager { kv_backend: KvBackendRef, } +pub fn flow_info_decoder(kv: KeyValue) -> Result<(FlowInfoKey, FlowInfoValue)> { + let key = FlowInfoKey::from_bytes(&kv.key)?; + let value = FlowInfoValue::try_from_raw_value(&kv.value)?; + Ok((key, value)) +} + impl FlowInfoManager { /// Returns a new [FlowInfoManager]. pub fn new(kv_backend: KvBackendRef) -> Self { @@ -254,6 +307,23 @@ impl FlowInfoManager { .transpose() } + pub fn flow_infos(&self) -> BoxStream<'static, Result<(FlowId, FlowInfoValue)>> { + let start_key = FlowScoped::new(BytesAdapter::from( + format!("{FLOW_INFO_KEY_PREFIX}/").into_bytes(), + )) + .to_bytes(); + let req = RangeRequest::new().with_prefix(start_key); + let stream = PaginationStream::new( + self.kv_backend.clone(), + req, + DEFAULT_PAGE_SIZE, + flow_info_decoder, + ) + .into_stream(); + + Box::pin(stream.map_ok(|(key, value)| (key.flow_id(), value))) + } + /// Builds a create flow transaction. /// It is expected that the `__flow/info/{flow_id}` wasn't occupied. /// Otherwise, the transaction will retrieve existing value. diff --git a/src/meta-srv/src/error.rs b/src/meta-srv/src/error.rs index a0f800f98164..dcb77f90f38f 100644 --- a/src/meta-srv/src/error.rs +++ b/src/meta-srv/src/error.rs @@ -135,6 +135,20 @@ pub enum Error { source: common_meta::error::Error, }, + #[snafu(display("Failed to init pending flow reconcile manager"))] + InitPendingFlowReconcileManager { + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + + #[snafu(display("Failed to reconcile pending flows"))] + PendingFlowReconcile { + #[snafu(implicit)] + location: Location, + source: common_meta::error::Error, + }, + #[snafu(display("Failed to create default catalog and schema"))] InitMetadata { #[snafu(implicit)] @@ -1277,6 +1291,8 @@ impl ErrorExt for Error { Error::InitMetadata { source, .. } | Error::InitDdlManager { source, .. } | Error::InitReconciliationManager { source, .. } => source.status_code(), + Error::InitPendingFlowReconcileManager { source, .. } => source.status_code(), + Error::PendingFlowReconcile { source, .. } => source.status_code(), Error::BuildTlsOptions { source, .. } => source.status_code(), Error::Other { source, .. } => source.status_code(), diff --git a/src/meta-srv/src/flow.rs b/src/meta-srv/src/flow.rs new file mode 100644 index 000000000000..8f7c22b9b1ab --- /dev/null +++ b/src/meta-srv/src/flow.rs @@ -0,0 +1,164 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; +use std::time::Duration; + +use common_meta::ddl_manager::DdlManagerRef; +use common_meta::key::table_name::TableNameKey; +use common_telemetry::{error, info}; +use futures::TryStreamExt; +use snafu::ResultExt; +use tokio::sync::mpsc::{Receiver, Sender}; + +use crate::define_ticker; +use crate::error::{PendingFlowReconcileSnafu, Result}; + +const PENDING_FLOW_RECONCILE_INTERVAL: Duration = Duration::from_secs(10); + +pub enum Event { + Tick, +} + +pub type PendingFlowReconcileTickerRef = Arc; + +define_ticker!( + PendingFlowReconcileTicker, + event_type = Event, + event_value = Event::Tick +); + +pub struct PendingFlowReconcileManager { + ddl_manager: DdlManagerRef, + receiver: Receiver, +} + +impl PendingFlowReconcileManager { + pub fn new(ddl_manager: DdlManagerRef) -> (Self, PendingFlowReconcileTicker) { + let (sender, receiver) = Self::channel(); + ( + Self { + ddl_manager, + receiver, + }, + PendingFlowReconcileTicker::new(PENDING_FLOW_RECONCILE_INTERVAL, sender), + ) + } + + fn channel() -> (Sender, Receiver) { + tokio::sync::mpsc::channel(8) + } + + pub fn try_start(mut self) -> Result<()> { + common_runtime::spawn_global(async move { self.run().await }); + info!("Pending flow reconcile manager started"); + Ok(()) + } + + async fn run(&mut self) { + while let Some(event) = self.receiver.recv().await { + match event { + Event::Tick => { + if let Err(e) = self.handle_tick().await { + error!(e; "Failed to reconcile pending flows"); + } + } + } + } + } + + async fn handle_tick(&self) -> Result<()> { + let ddl_context = self.ddl_manager.create_context(); + let ddl_manager = self.ddl_manager.clone(); + ddl_context + .flow_metadata_manager + .flow_info_manager() + .flow_infos() + .try_for_each(move |(flow_id, flow_info)| { + let ddl_context = ddl_context.clone(); + let ddl_manager = ddl_manager.clone(); + async move { + if !flow_info.is_pending() { + return Ok(()); + } + + let current_flow_info = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get_raw(flow_id) + .await; + let current_flow_info = match current_flow_info { + Ok(current_flow_info) => current_flow_info, + Err(e) => { + error!(e; "Failed to load flow metadata for pending flow {}", flow_id); + return Ok(()); + } + }; + let Some(current_flow_info) = current_flow_info else { + return Ok(()); + }; + if !current_flow_info.get_inner_ref().is_pending() { + return Ok(()); + } + + let unresolved_source_table_names = current_flow_info + .get_inner_ref() + .unresolved_source_table_names(); + if !unresolved_source_table_names.is_empty() { + let unresolved_table_keys = unresolved_source_table_names + .iter() + .map(|name| { + TableNameKey::new( + &name.catalog_name, + &name.schema_name, + &name.table_name, + ) + }) + .collect::>(); + let resolved_tables = ddl_context + .table_metadata_manager + .table_name_manager() + .batch_get(unresolved_table_keys) + .await; + let resolved_tables = match resolved_tables { + Ok(resolved_tables) => resolved_tables, + Err(e) => { + error!(e; "Failed to resolve source tables for pending flow {}", flow_id); + return Ok(()); + } + }; + if resolved_tables.iter().all(|table_id| table_id.is_none()) { + return Ok(()); + } + } + + if let Err(e) = ddl_manager + .submit_activate_pending_flow_task( + flow_id, + current_flow_info.get_inner_ref().catalog_name().clone(), + ) + .await + { + error!(e; "Failed to reconcile pending flow {}", flow_id); + } + + Ok(()) + } + }) + .await + .context(PendingFlowReconcileSnafu)?; + + Ok(()) + } +} diff --git a/src/meta-srv/src/lib.rs b/src/meta-srv/src/lib.rs index 70ce449bba80..c9045c4df98c 100644 --- a/src/meta-srv/src/lib.rs +++ b/src/meta-srv/src/lib.rs @@ -22,6 +22,7 @@ pub mod discovery; pub mod error; pub mod events; mod failure_detector; +pub mod flow; pub mod gc; pub mod handler; pub mod key; diff --git a/src/meta-srv/src/metasrv.rs b/src/meta-srv/src/metasrv.rs index 654457814888..6acb1b8076ed 100644 --- a/src/meta-srv/src/metasrv.rs +++ b/src/meta-srv/src/metasrv.rs @@ -71,6 +71,7 @@ use crate::error::{ StartTelemetryTaskSnafu, StopProcedureManagerSnafu, }; use crate::failure_detector::PhiAccrualFailureDetectorOptions; +use crate::flow::PendingFlowReconcileTickerRef; use crate::gc::{GcSchedulerOptions, GcTickerRef}; use crate::handler::{HeartbeatHandlerGroupBuilder, HeartbeatHandlerGroupRef}; use crate::procedure::ProcedureManagerListenerAdapter; @@ -569,6 +570,7 @@ pub struct Metasrv { region_flush_ticker: Option, table_id_allocator: ResourceIdAllocatorRef, reconciliation_manager: ReconciliationManagerRef, + pending_flow_reconcile_ticker: Option, resource_stat: ResourceStatRef, gc_ticker: Option, database_operator: DatabaseOperatorRef, @@ -635,6 +637,9 @@ impl Metasrv { if let Some(gc_ticker) = &self.gc_ticker { leadership_change_notifier.add_listener(gc_ticker.clone() as _); } + if let Some(pending_flow_reconcile_ticker) = &self.pending_flow_reconcile_ticker { + leadership_change_notifier.add_listener(pending_flow_reconcile_ticker.clone() as _); + } if let Some(customizer) = self.plugins.get::() { customizer.customize(&mut leadership_change_notifier); } diff --git a/src/meta-srv/src/metasrv/builder.rs b/src/meta-srv/src/metasrv/builder.rs index 54a94fffbf52..d30775f1a49f 100644 --- a/src/meta-srv/src/metasrv/builder.rs +++ b/src/meta-srv/src/metasrv/builder.rs @@ -56,6 +56,7 @@ use crate::cache_invalidator::MetasrvCacheInvalidator; use crate::cluster::MetaPeerClientRef; use crate::error::{self, BuildWalProviderSnafu, OtherSnafu, Result}; use crate::events::EventHandlerImpl; +use crate::flow::{PendingFlowReconcileManager, PendingFlowReconcileTickerRef}; use crate::gc::GcScheduler; use crate::greptimedb_telemetry::get_greptimedb_telemetry_task; use crate::handler::failure_handler::RegionFailureHandler; @@ -558,6 +559,15 @@ impl MetasrvBuilder { .try_start() .context(error::InitReconciliationManagerSnafu)?; + let (pending_flow_reconcile_manager, pending_flow_reconcile_ticker) = + PendingFlowReconcileManager::new(ddl_manager.clone()); + pending_flow_reconcile_manager + .try_start() + .map_err(common_error::ext::BoxedError::new) + .context(error::InitPendingFlowReconcileManagerSnafu)?; + let pending_flow_reconcile_ticker: Option = + Some(Arc::new(pending_flow_reconcile_ticker)); + let mut resource_stat = ResourceStatImpl::default(); resource_stat.start_collect_cpu_usage(); @@ -599,6 +609,7 @@ impl MetasrvBuilder { region_flush_ticker, table_id_allocator, reconciliation_manager, + pending_flow_reconcile_ticker, topic_stats_registry, resource_stat: Arc::new(resource_stat), gc_ticker, diff --git a/src/operator/src/statement/ddl.rs b/src/operator/src/statement/ddl.rs index 6fd6e4adb431..f8d40271fffc 100644 --- a/src/operator/src/statement/ddl.rs +++ b/src/operator/src/statement/ddl.rs @@ -204,6 +204,39 @@ fn validate_and_normalize_flow_options( .collect() } +fn determine_flow_type_for_source_state( + flow_name: &str, + flow_options: &HashMap, + has_missing_source_table: bool, + has_instant_ttl_source_table: bool, +) -> Result> { + if has_missing_source_table { + let defer_on_missing_source = flow_options + .get(DEFER_ON_MISSING_SOURCE_KEY) + .is_some_and(|value| value == "true"); + ensure!( + defer_on_missing_source, + InvalidSqlSnafu { + err_msg: format!( + "missing source tables for flow '{}'; use WITH ({DEFER_ON_MISSING_SOURCE_KEY} = true) to create a pending flow", + flow_name + ) + } + ); + info!( + "Flow `{}` defaults to batching because defer_on_missing_source=true and some source tables are not available yet", + flow_name + ); + return Ok(Some(FlowType::Batching)); + } + + if has_instant_ttl_source_table { + return Ok(Some(FlowType::Streaming)); + } + + Ok(None) +} + impl StatementExecutor { pub fn catalog_manager(&self) -> CatalogManagerRef { self.catalog_manager.clone() @@ -714,7 +747,9 @@ impl StatementExecutor { expr: &CreateFlowExpr, query_ctx: QueryContextRef, ) -> Result { - // first check if source table's ttl is instant, if it is, force streaming mode + let mut has_missing_source_table = false; + let mut has_instant_ttl_source_table = false; + for src_table_name in &expr.source_table_names { let table = self .catalog_manager() @@ -726,16 +761,13 @@ impl StatementExecutor { ) .await .map_err(BoxedError::new) - .context(ExternalSnafu)? - .with_context(|| TableNotFoundSnafu { - table_name: format_full_table_name( - &src_table_name.catalog_name, - &src_table_name.schema_name, - &src_table_name.table_name, - ), - })?; + .context(ExternalSnafu)?; + + let Some(table) = table else { + has_missing_source_table = true; + continue; + }; - // instant source table can only be handled by streaming mode if table.table_info().meta.options.ttl == Some(common_time::TimeToLive::Instant) { warn!( "Source table `{}` for flow `{}`'s ttl=instant, fallback to streaming mode", @@ -746,10 +778,19 @@ impl StatementExecutor { ), expr.flow_name ); - return Ok(FlowType::Streaming); + has_instant_ttl_source_table = true; } } + if let Some(flow_type) = determine_flow_type_for_source_state( + &expr.flow_name, + &expr.flow_options, + has_missing_source_table, + has_instant_ttl_source_table, + )? { + return Ok(flow_type); + } + let engine = &self.query_engine; let stmts = ParserContext::create_with_dialect( &expr.sql, @@ -2472,6 +2513,35 @@ SELECT max(c1), min(c2) FROM schema_2.table_2;"; )); } + #[test] + fn test_determine_flow_type_for_source_state_missing_sources_require_opt_in() { + let err = determine_flow_type_for_source_state("my_flow", &HashMap::new(), true, false) + .unwrap_err(); + + assert!(err.to_string().contains( + "missing source tables for flow 'my_flow'; use WITH (defer_on_missing_source = true) to create a pending flow" + )); + } + + #[test] + fn test_determine_flow_type_for_source_state_missing_sources_prefer_batching() { + let flow_options = + HashMap::from([(DEFER_ON_MISSING_SOURCE_KEY.to_string(), "true".to_string())]); + + assert_eq!( + determine_flow_type_for_source_state("my_flow", &flow_options, true, true).unwrap(), + Some(FlowType::Batching) + ); + } + + #[test] + fn test_determine_flow_type_for_source_state_instant_ttl_without_missing_sources() { + assert_eq!( + determine_flow_type_for_source_state("my_flow", &HashMap::new(), false, true).unwrap(), + Some(FlowType::Streaming) + ); + } + #[test] fn test_name_is_match() { assert!(!NAME_PATTERN_REG.is_match("/adaf")); diff --git a/tests/cases/standalone/common/flow/flow_pending.result b/tests/cases/standalone/common/flow/flow_pending.result new file mode 100644 index 000000000000..80a56cb22b44 --- /dev/null +++ b/tests/cases/standalone/common/flow/flow_pending.result @@ -0,0 +1,148 @@ +CREATE FLOW pending_without_defer +SINK TO pending_sink +AS SELECT val FROM pending_source; + +Error: 1004(InvalidArguments), Invalid SQL, error: missing source tables for flow 'pending_without_defer'; use WITH (defer_on_missing_source = true) to create a pending flow + +CREATE FLOW pending_with_defer +SINK TO pending_sink +WITH (defer_on_missing_source = true) +AS SELECT val FROM pending_source WHERE val > 10; + +Affected Rows: 0 + +SHOW CREATE FLOW pending_with_defer; + ++--------------------+--------------------------------------------------+ +| Flow | Create Flow | ++--------------------+--------------------------------------------------+ +| pending_with_defer | CREATE FLOW IF NOT EXISTS pending_with_defer | +| | SINK TO public.pending_sink | +| | WITH (defer_on_missing_source = 'true') | +| | AS SELECT val FROM pending_source WHERE val > 10 | ++--------------------+--------------------------------------------------+ + +SELECT + flow_definition, + source_table_ids, + source_table_names, + flownode_ids, + options LIKE '%"defer_on_missing_source":"true"%' AS has_defer_option, + options LIKE '%"flow_type":"batching"%' AS has_flow_type_option +FROM INFORMATION_SCHEMA.FLOWS +WHERE flow_name = 'pending_with_defer'; + ++--------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ +| flow_definition | source_table_ids | source_table_names | flownode_ids | has_defer_option | has_flow_type_option | ++--------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ +| CREATE FLOW IF NOT EXISTS pending_with_defer | [] | | {} | true | true | +| SINK TO public.pending_sink | | | | | | +| WITH (defer_on_missing_source = 'true') | | | | | | +| AS SELECT val FROM pending_source WHERE val > 10 | | | | | | ++--------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ + +CREATE TABLE pending_source ( + val INT, + ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, + TIME INDEX(ts) +); + +Affected Rows: 0 + +-- SQLNESS SLEEP 12s +SELECT + flow_definition, + source_table_ids, + source_table_names, + flownode_ids, + options LIKE '%"defer_on_missing_source":"true"%' AS has_defer_option, + options LIKE '%"flow_type":"batching"%' AS has_flow_type_option +FROM INFORMATION_SCHEMA.FLOWS +WHERE flow_name = 'pending_with_defer'; + ++--------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ +| flow_definition | source_table_ids | source_table_names | flownode_ids | has_defer_option | has_flow_type_option | ++--------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ +| CREATE FLOW IF NOT EXISTS pending_with_defer | [] | | {} | true | true | +| SINK TO public.pending_sink | | | | | | +| WITH (defer_on_missing_source = 'true') | | | | | | +| AS SELECT val FROM pending_source WHERE val > 10 | | | | | | ++--------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ + +INSERT INTO pending_source VALUES (10, 0), (11, 1), (12, 2); + +Affected Rows: 3 + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('pending_with_defer'); + +Error: 1002(Unexpected), Failed to execute admin function flush_flow: Execution error: Can't found alive flownode + +SELECT val FROM pending_sink ORDER BY val; + +Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.public.pending_sink + +CREATE OR REPLACE FLOW pending_with_defer +SINK TO pending_sink +WITH (defer_on_missing_source = true) +AS SELECT val FROM pending_replacement_source WHERE val > 100; + +Affected Rows: 0 + +SELECT + flow_definition, + source_table_ids, + source_table_names, + flownode_ids, + options LIKE '%"defer_on_missing_source":"true"%' AS has_defer_option, + options LIKE '%"flow_type":"batching"%' AS has_flow_type_option +FROM INFORMATION_SCHEMA.FLOWS +WHERE flow_name = 'pending_with_defer'; + ++---------------------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ +| flow_definition | source_table_ids | source_table_names | flownode_ids | has_defer_option | has_flow_type_option | ++---------------------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ +| CREATE FLOW IF NOT EXISTS pending_with_defer | [] | | {} | true | true | +| SINK TO public.pending_sink | | | | | | +| WITH (defer_on_missing_source = 'true') | | | | | | +| AS SELECT val FROM pending_replacement_source WHERE val > 100 | | | | | | ++---------------------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ + +CREATE TABLE pending_replacement_source ( + val INT, + ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, + TIME INDEX(ts) +); + +Affected Rows: 0 + +-- SQLNESS SLEEP 12s +INSERT INTO pending_replacement_source VALUES (99, 3), (101, 4), (102, 5); + +Affected Rows: 3 + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('pending_with_defer'); + +Error: 1002(Unexpected), Failed to execute admin function flush_flow: Execution error: Can't found alive flownode + +SELECT val FROM pending_sink ORDER BY val; + +Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.public.pending_sink + +DROP FLOW pending_with_defer; + +Error: 1002(Unexpected), Flow route not found: 'greptime.pending_with_defer' + +DROP TABLE pending_sink; + +Error: 4001(TableNotFound), Table not found: greptime.public.pending_sink + +DROP TABLE pending_source; + +Affected Rows: 0 + +DROP TABLE pending_replacement_source; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/common/flow/flow_pending.sql b/tests/cases/standalone/common/flow/flow_pending.sql new file mode 100644 index 000000000000..d74de54446fa --- /dev/null +++ b/tests/cases/standalone/common/flow/flow_pending.sql @@ -0,0 +1,83 @@ +CREATE FLOW pending_without_defer +SINK TO pending_sink +AS SELECT val FROM pending_source; + +CREATE FLOW pending_with_defer +SINK TO pending_sink +WITH (defer_on_missing_source = true) +AS SELECT val FROM pending_source WHERE val > 10; + +SHOW CREATE FLOW pending_with_defer; + +SELECT + flow_definition, + source_table_ids, + source_table_names, + flownode_ids, + options LIKE '%"defer_on_missing_source":"true"%' AS has_defer_option, + options LIKE '%"flow_type":"batching"%' AS has_flow_type_option +FROM INFORMATION_SCHEMA.FLOWS +WHERE flow_name = 'pending_with_defer'; + +CREATE TABLE pending_source ( + val INT, + ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, + TIME INDEX(ts) +); + +-- SQLNESS SLEEP 12s + +SELECT + flow_definition, + source_table_ids, + source_table_names, + flownode_ids, + options LIKE '%"defer_on_missing_source":"true"%' AS has_defer_option, + options LIKE '%"flow_type":"batching"%' AS has_flow_type_option +FROM INFORMATION_SCHEMA.FLOWS +WHERE flow_name = 'pending_with_defer'; + +INSERT INTO pending_source VALUES (10, 0), (11, 1), (12, 2); + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('pending_with_defer'); + +SELECT val FROM pending_sink ORDER BY val; + +CREATE OR REPLACE FLOW pending_with_defer +SINK TO pending_sink +WITH (defer_on_missing_source = true) +AS SELECT val FROM pending_replacement_source WHERE val > 100; + +SELECT + flow_definition, + source_table_ids, + source_table_names, + flownode_ids, + options LIKE '%"defer_on_missing_source":"true"%' AS has_defer_option, + options LIKE '%"flow_type":"batching"%' AS has_flow_type_option +FROM INFORMATION_SCHEMA.FLOWS +WHERE flow_name = 'pending_with_defer'; + +CREATE TABLE pending_replacement_source ( + val INT, + ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, + TIME INDEX(ts) +); + +-- SQLNESS SLEEP 12s + +INSERT INTO pending_replacement_source VALUES (99, 3), (101, 4), (102, 5); + +-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | +ADMIN FLUSH_FLOW('pending_with_defer'); + +SELECT val FROM pending_sink ORDER BY val; + +DROP FLOW pending_with_defer; + +DROP TABLE pending_sink; + +DROP TABLE pending_source; + +DROP TABLE pending_replacement_source; From f154e18b159f399814785af0fad9482e841dde63 Mon Sep 17 00:00:00 2001 From: discord9 Date: Tue, 19 May 2026 22:37:34 +0800 Subject: [PATCH 2/4] chore: simplify pending flow PR scope Reduce PR #8124 to the metadata-only MVP after complexity review. Changes: - Remove automatic activation procedure and meta-srv reconcile wiring - Remove activation tests and activation-only metadata fields - Reject cross-state pending<->active `OR REPLACE` transitions for now - Keep pending metadata creation and SQL round-trip behavior - Allow `DROP FLOW` for pending flows without routes - Reduce flow_pending sqlness to metadata/round-trip/drop coverage only Deferred follow-ups are documented locally in `.tmp/tasks/pending-defer-semantics/deferred-followups.md` and intentionally not committed. Tests: - `cargo test -p operator determine_flow_type_for_source_state` - `cargo test -p common-meta create_flow` - `cargo test -p common-meta drop_flow` - `cargo sqlness bare --test-filter flow_pending --bins-dir /mnt/nvme_rust/rust-targets/pending_defer/debug` Signed-off-by: discord9 --- .../meta/src/cache/flow/table_flownode.rs | 1 - src/common/meta/src/ddl.rs | 1 - src/common/meta/src/ddl/activate_flow.rs | 468 ------------------ src/common/meta/src/ddl/create_flow.rs | 75 +-- src/common/meta/src/ddl/drop_flow/metadata.rs | 2 +- src/common/meta/src/ddl/flow_meta.rs | 7 +- src/common/meta/src/ddl/tests.rs | 1 - .../meta/src/ddl/tests/activate_flow.rs | 329 ------------ src/common/meta/src/ddl/tests/create_flow.rs | 106 +--- src/common/meta/src/ddl/tests/drop_flow.rs | 44 +- src/common/meta/src/ddl_manager.rs | 16 - src/common/meta/src/key/flow.rs | 3 - src/common/meta/src/key/flow/flow_info.rs | 6 - src/meta-srv/src/error.rs | 16 - src/meta-srv/src/flow.rs | 164 ------ src/meta-srv/src/lib.rs | 1 - src/meta-srv/src/metasrv.rs | 5 - src/meta-srv/src/metasrv/builder.rs | 11 - .../common/flow/flow_pending.result | 102 +--- .../standalone/common/flow/flow_pending.sql | 61 +-- 20 files changed, 89 insertions(+), 1330 deletions(-) delete mode 100644 src/common/meta/src/ddl/activate_flow.rs delete mode 100644 src/common/meta/src/ddl/tests/activate_flow.rs delete mode 100644 src/meta-srv/src/flow.rs diff --git a/src/common/meta/src/cache/flow/table_flownode.rs b/src/common/meta/src/cache/flow/table_flownode.rs index 8cd5d8b66233..4d3513a21dbf 100644 --- a/src/common/meta/src/cache/flow/table_flownode.rs +++ b/src/common/meta/src/cache/flow/table_flownode.rs @@ -250,7 +250,6 @@ mod tests { comment: "comment".to_string(), options: Default::default(), status: FlowStatus::Active, - last_activation_error: None, created_time: chrono::Utc::now(), updated_time: chrono::Utc::now(), }, diff --git a/src/common/meta/src/ddl.rs b/src/common/meta/src/ddl.rs index 42fd8ffe7669..8fc647433a3c 100644 --- a/src/common/meta/src/ddl.rs +++ b/src/common/meta/src/ddl.rs @@ -28,7 +28,6 @@ use crate::node_manager::NodeManagerRef; use crate::region_keeper::MemoryRegionKeeperRef; use crate::region_registry::LeaderRegionRegistryRef; -pub mod activate_flow; pub mod allocator; pub mod alter_database; pub mod alter_logical_tables; diff --git a/src/common/meta/src/ddl/activate_flow.rs b/src/common/meta/src/ddl/activate_flow.rs deleted file mode 100644 index 86a1f5fec3f9..000000000000 --- a/src/common/meta/src/ddl/activate_flow.rs +++ /dev/null @@ -1,468 +0,0 @@ -// Copyright 2023 Greptime Team -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::collections::BTreeMap; - -use api::v1::ExpireAfter; -use api::v1::flow::flow_request::Body as PbFlowRequest; -use api::v1::flow::{CreateRequest, FlowRequest, FlowRequestHeader}; -use async_trait::async_trait; -use common_procedure::error::{FromJsonSnafu, ToJsonSnafu}; -use common_procedure::{ - Context as ProcedureContext, LockKey, Procedure, Result as ProcedureResult, Status, -}; -use common_telemetry::tracing_context::TracingContext; -use futures::future::join_all; -use itertools::Itertools; -use serde::{Deserialize, Serialize}; -use snafu::{OptionExt, ResultExt}; -use table::metadata::TableId; -use table::table_name::TableName; - -use crate::cache_invalidator::Context; -use crate::ddl::DdlContext; -use crate::ddl::create_flow::{DEFER_ON_MISSING_SOURCE_KEY, FlowType}; -use crate::ddl::utils::{add_peer_context_if_needed, map_to_procedure_error}; -use crate::error::{self, Result}; -use crate::instruction::{CacheIdent, CreateFlow}; -use crate::key::flow::flow_info::{FlowInfoValue, FlowStatus}; -use crate::key::flow::flow_route::FlowRouteValue; -use crate::key::table_name::TableNameKey; -use crate::key::{DeserializedValueWithBytes, FlowId, FlowPartitionId}; -use crate::lock_key::{CatalogLock, FlowLock, FlowNameLock}; -use crate::metrics; -use crate::peer::Peer; - -/// Activates a pending flow after all source tables become available. -pub struct ActivatePendingFlowProcedure { - pub context: DdlContext, - pub data: ActivatePendingFlowData, -} - -#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] -pub enum ActivatePendingFlowState { - Prepare, - CreateFlows, - UpdateMetadata, - InvalidateFlowCache, -} - -#[derive(Debug, Serialize, Deserialize)] -pub struct ActivatePendingFlowData { - pub(crate) state: ActivatePendingFlowState, - pub(crate) flow_id: FlowId, - pub(crate) catalog_name: String, - #[serde(default)] - pub(crate) peers: Vec, - #[serde(default)] - pub(crate) resolved_table_ids: Vec, - pub(crate) prev_flow_info_value: Option>, -} - -pub struct PendingFlowResolution { - resolved_table_ids: Vec, - unresolved_source_table_names: Vec, -} - -impl ActivatePendingFlowProcedure { - pub const TYPE_NAME: &'static str = "metasrv-procedure::ActivatePendingFlow"; - - pub fn new(flow_id: FlowId, catalog_name: String, context: DdlContext) -> Self { - Self { - context, - data: ActivatePendingFlowData { - state: ActivatePendingFlowState::Prepare, - flow_id, - catalog_name, - peers: vec![], - resolved_table_ids: vec![], - prev_flow_info_value: None, - }, - } - } - - pub fn from_json(json: &str, context: DdlContext) -> ProcedureResult { - let data = serde_json::from_str(json).context(FromJsonSnafu)?; - Ok(Self { context, data }) - } - - async fn on_prepare(&mut self) -> Result { - let Some(current_flow_info) = self - .context - .flow_metadata_manager - .flow_info_manager() - .get_raw(self.data.flow_id) - .await? - else { - common_telemetry::debug!( - "Pending flow {} no longer exists during activation", - self.data.flow_id - ); - return Ok(Status::done()); - }; - - if current_flow_info.get_inner_ref().is_active() { - let routes = self - .context - .flow_metadata_manager - .flow_route_manager() - .routes(self.data.flow_id) - .await?; - self.data.peers = routes.into_iter().map(|(_, value)| value.peer).collect(); - self.data.resolved_table_ids = - current_flow_info.get_inner_ref().source_table_ids.clone(); - self.data.state = ActivatePendingFlowState::InvalidateFlowCache; - return Ok(Status::executing(true)); - } - - let resolution = - resolve_pending_flow_sources(&self.context, current_flow_info.get_inner_ref()).await?; - common_telemetry::debug!( - "Resolved pending flow {} source tables: {} resolved, {} unresolved", - self.data.flow_id, - resolution.resolved_table_ids.len(), - resolution.unresolved_source_table_names.len() - ); - if !resolution.unresolved_source_table_names.is_empty() { - update_pending_flow_metadata( - &self.context, - self.data.flow_id, - ¤t_flow_info, - resolution.resolved_table_ids, - resolution.unresolved_source_table_names, - None, - ) - .await?; - return Ok(Status::done()); - } - - self.data.peers = self.context.flow_metadata_allocator.alloc_peers(1).await?; - self.data.resolved_table_ids = resolution.resolved_table_ids; - self.data.prev_flow_info_value = Some(current_flow_info); - self.data.state = ActivatePendingFlowState::CreateFlows; - - Ok(Status::executing(true)) - } - - async fn on_create_flows(&mut self) -> Result { - let flow_info = - self.data - .prev_flow_info_value - .as_ref() - .context(error::UnexpectedSnafu { - err_msg: "missing previous flow info for activation", - })?; - let request = build_create_request( - self.data.flow_id, - flow_info.get_inner_ref(), - &self.data.resolved_table_ids, - ); - debug_assert_eq!( - self.data.resolved_table_ids.len(), - flow_info.get_inner_ref().all_source_table_names().len(), - "All source tables must be resolved before pending flow activation" - ); - create_flow_on_peers( - &self.context, - &self.data.peers, - request, - flow_info.get_inner_ref(), - ) - .await?; - self.data.state = ActivatePendingFlowState::UpdateMetadata; - Ok(Status::executing(true)) - } - - async fn on_update_metadata(&mut self) -> Result { - let current_flow_info = - self.data - .prev_flow_info_value - .as_ref() - .context(error::UnexpectedSnafu { - err_msg: "missing previous flow info for metadata update", - })?; - let (new_flow_info, flow_routes) = build_active_flow_info( - current_flow_info.get_inner_ref(), - &self.data.peers, - &self.data.resolved_table_ids, - ); - self.context - .flow_metadata_manager - .update_flow_metadata( - self.data.flow_id, - current_flow_info, - &new_flow_info, - flow_routes, - ) - .await?; - self.data.state = ActivatePendingFlowState::InvalidateFlowCache; - Ok(Status::executing(true)) - } - - async fn on_broadcast(&mut self) -> Result { - invalidate_flow_cache( - &self.context, - self.data.flow_id, - &self.data.resolved_table_ids, - &self.data.peers, - ) - .await - .map_err(error::Error::retry_later)?; - Ok(Status::done_with_output(self.data.flow_id)) - } -} - -#[async_trait] -impl Procedure for ActivatePendingFlowProcedure { - fn type_name(&self) -> &str { - Self::TYPE_NAME - } - - async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult { - let state = &self.data.state; - let _timer = metrics::METRIC_META_PROCEDURE_CREATE_FLOW - .with_label_values(&[match state { - ActivatePendingFlowState::Prepare => "activate_prepare", - ActivatePendingFlowState::CreateFlows => "activate_create_flows", - ActivatePendingFlowState::UpdateMetadata => "activate_update_metadata", - ActivatePendingFlowState::InvalidateFlowCache => "activate_invalidate_flow_cache", - }]) - .start_timer(); - - match self.data.state { - ActivatePendingFlowState::Prepare => self.on_prepare().await, - ActivatePendingFlowState::CreateFlows => self.on_create_flows().await, - ActivatePendingFlowState::UpdateMetadata => self.on_update_metadata().await, - ActivatePendingFlowState::InvalidateFlowCache => self.on_broadcast().await, - } - .map_err(map_to_procedure_error) - } - - fn dump(&self) -> ProcedureResult { - serde_json::to_string(&self.data).context(ToJsonSnafu) - } - - fn lock_key(&self) -> LockKey { - if let Some(flow_info) = &self.data.prev_flow_info_value { - LockKey::new(vec![ - CatalogLock::Read(&self.data.catalog_name).into(), - FlowNameLock::new( - flow_info.get_inner_ref().catalog_name(), - flow_info.get_inner_ref().flow_name(), - ) - .into(), - FlowLock::Write(self.data.flow_id).into(), - ]) - } else { - LockKey::new(vec![ - CatalogLock::Read(&self.data.catalog_name).into(), - FlowLock::Write(self.data.flow_id).into(), - ]) - } - } -} - -async fn resolve_pending_flow_sources( - context: &DdlContext, - flow_info: &FlowInfoValue, -) -> Result { - let keys = flow_info - .all_source_table_names() - .iter() - .map(|name| TableNameKey::new(&name.catalog_name, &name.schema_name, &name.table_name)) - .collect::>(); - let resolved_tables = context - .table_metadata_manager - .table_name_manager() - .batch_get(keys) - .await?; - - let mut resolved_table_ids = Vec::with_capacity(flow_info.all_source_table_names().len()); - let mut unresolved_source_table_names = Vec::new(); - for (name, table_id) in flow_info - .all_source_table_names() - .iter() - .zip(resolved_tables) - { - match table_id { - Some(table_id) => resolved_table_ids.push(table_id.table_id()), - None => unresolved_source_table_names.push(name.clone()), - } - } - - Ok(PendingFlowResolution { - resolved_table_ids, - unresolved_source_table_names, - }) -} - -async fn update_pending_flow_metadata( - context: &DdlContext, - flow_id: FlowId, - current_flow_info: &DeserializedValueWithBytes, - resolved_table_ids: Vec, - unresolved_source_table_names: Vec, - last_activation_error: Option, -) -> Result<()> { - let current_flow_info_value = current_flow_info.get_inner_ref(); - if current_flow_info_value.source_table_ids == resolved_table_ids - && current_flow_info_value.unresolved_source_table_names == unresolved_source_table_names - && current_flow_info_value.last_activation_error == last_activation_error - { - return Ok(()); - } - - let mut new_flow_info = current_flow_info.get_inner_ref().clone(); - new_flow_info.source_table_ids = resolved_table_ids; - new_flow_info.unresolved_source_table_names = unresolved_source_table_names; - new_flow_info.last_activation_error = last_activation_error; - new_flow_info.updated_time = chrono::Utc::now(); - context - .flow_metadata_manager - .update_flow_metadata(flow_id, current_flow_info, &new_flow_info, vec![]) - .await -} - -fn build_create_request( - flow_id: FlowId, - flow_info: &FlowInfoValue, - resolved_table_ids: &[TableId], -) -> CreateRequest { - let mut flow_options = flow_info.options.clone(); - flow_options.remove(DEFER_ON_MISSING_SOURCE_KEY); - flow_options.insert( - FlowType::FLOW_TYPE_KEY.to_string(), - get_flow_type(flow_info).to_string(), - ); - CreateRequest { - flow_id: Some(api::v1::FlowId { id: flow_id }), - source_table_ids: resolved_table_ids - .iter() - .map(|table_id| api::v1::TableId { id: *table_id }) - .collect_vec(), - sink_table_name: Some(flow_info.sink_table_name.clone().into()), - create_if_not_exists: false, - or_replace: true, - expire_after: flow_info.expire_after.map(|value| ExpireAfter { value }), - eval_interval: flow_info - .eval_interval_secs - .map(|seconds| api::v1::EvalInterval { seconds }), - comment: flow_info.comment.clone(), - sql: flow_info.raw_sql.clone(), - flow_options, - } -} - -async fn create_flow_on_peers( - context: &DdlContext, - peers: &[Peer], - request: CreateRequest, - flow_info: &FlowInfoValue, -) -> Result<()> { - let query_context = flow_info.query_context.clone().unwrap_or_default(); - let mut create_flow_tasks = Vec::with_capacity(peers.len()); - for peer in peers { - let requester = context.node_manager.flownode(peer).await; - let request = FlowRequest { - header: Some(FlowRequestHeader { - tracing_context: TracingContext::from_current_span().to_w3c(), - query_context: Some(query_context.clone().into()), - }), - body: Some(PbFlowRequest::Create(request.clone())), - }; - create_flow_tasks.push(async move { - requester - .handle(request) - .await - .map_err(add_peer_context_if_needed(peer.clone())) - }); - } - join_all(create_flow_tasks) - .await - .into_iter() - .collect::>>()?; - Ok(()) -} - -fn build_active_flow_info( - current_flow_info: &FlowInfoValue, - peers: &[Peer], - resolved_table_ids: &[TableId], -) -> (FlowInfoValue, Vec<(FlowPartitionId, FlowRouteValue)>) { - let flow_type = get_flow_type(current_flow_info); - let flownode_ids = peers - .iter() - .enumerate() - .map(|(idx, peer)| (idx as u32, peer.id)) - .collect::>(); - let flow_routes = peers - .iter() - .enumerate() - .map(|(idx, peer)| (idx as u32, FlowRouteValue { peer: peer.clone() })) - .collect::>(); - - let mut new_flow_info = current_flow_info.clone(); - new_flow_info.source_table_ids = resolved_table_ids.to_vec(); - new_flow_info.unresolved_source_table_names = vec![]; - new_flow_info.flownode_ids = flownode_ids; - new_flow_info.status = FlowStatus::Active; - new_flow_info.last_activation_error = None; - new_flow_info - .options - .insert(FlowType::FLOW_TYPE_KEY.to_string(), flow_type.to_string()); - new_flow_info.updated_time = chrono::Utc::now(); - - (new_flow_info, flow_routes) -} - -fn get_flow_type(flow_info: &FlowInfoValue) -> FlowType { - match flow_info - .options() - .get(FlowType::FLOW_TYPE_KEY) - .map(String::as_str) - { - Some(FlowType::STREAMING) => FlowType::Streaming, - _ => FlowType::Batching, - } -} - -async fn invalidate_flow_cache( - context: &DdlContext, - flow_id: FlowId, - resolved_table_ids: &[TableId], - peers: &[Peer], -) -> Result<()> { - let ctx = Context { - subject: Some("Invalidate flow cache by activating pending flow".to_string()), - }; - let flow_part2peers = peers - .iter() - .enumerate() - .map(|(idx, peer)| (idx as u32, peer.clone())) - .collect(); - context - .cache_invalidator - .invalidate( - &ctx, - &[ - CacheIdent::CreateFlow(CreateFlow { - flow_id, - source_table_ids: resolved_table_ids.to_vec(), - partition_to_peer_mapping: flow_part2peers, - }), - CacheIdent::FlowId(flow_id), - ], - ) - .await -} diff --git a/src/common/meta/src/ddl/create_flow.rs b/src/common/meta/src/ddl/create_flow.rs index 1265d09a5029..a4fe5b62ec70 100644 --- a/src/common/meta/src/ddl/create_flow.rs +++ b/src/common/meta/src/ddl/create_flow.rs @@ -19,7 +19,7 @@ use std::fmt; use api::v1::ExpireAfter; use api::v1::flow::flow_request::Body as PbFlowRequest; -use api::v1::flow::{CreateRequest, DropRequest, FlowRequest, FlowRequestHeader, flow_request}; +use api::v1::flow::{CreateRequest, FlowRequest, FlowRequestHeader}; use async_trait::async_trait; use common_catalog::format_full_flow_name; use common_procedure::error::{FromJsonSnafu, ToJsonSnafu}; @@ -72,10 +72,8 @@ impl CreateFlowProcedure { flow_context: query_context.into(), // Convert to FlowQueryContext state: CreateFlowState::Prepare, prev_flow_info_value: None, - prev_peers: vec![], did_replace: false, flow_type: None, - last_activation_error: None, }, } } @@ -133,8 +131,7 @@ impl CreateFlowProcedure { .map(|(_, value)| value.peer) .collect::>(); self.data.flow_id = Some(flow_id); - self.data.peers.clone_from(&peers); - self.data.prev_peers = peers; + self.data.peers = peers; info!("Replacing flow, flow_id: {}", flow_id); let flow_info_value = self @@ -186,6 +183,7 @@ impl CreateFlowProcedure { ) } ); + self.ensure_supported_replace_transition()?; // Validate that source and sink tables are not the same let sink_table_name = &self.data.task.sink_table_name; @@ -214,17 +212,32 @@ impl CreateFlowProcedure { self.data.peers.clear(); CreateFlowState::CreateMetadata } else { - if self.data.peers.is_empty() { - // Replacing a pending flow means the flow id already exists, but there - // are no flow routes yet. Allocate peers for the newly-active flow. - self.data.peers = self.context.flow_metadata_allocator.alloc_peers(1).await?; - } CreateFlowState::CreateFlows }; Ok(Status::executing(true)) } + fn ensure_supported_replace_transition(&self) -> Result<()> { + if !self.data.task.or_replace { + return Ok(()); + } + + let Some(prev_flow_info) = self.data.prev_flow_info_value.as_ref() else { + return Ok(()); + }; + let prev_pending = prev_flow_info.get_inner_ref().is_pending(); + let new_pending = self.data.is_pending(); + ensure!( + prev_pending == new_pending, + error::UnsupportedSnafu { + operation: "Replacing between pending and active flow states is not supported yet" + } + ); + + Ok(()) + } + async fn on_flownode_create_flows(&mut self) -> Result { // Safety: must be allocated. let mut create_flow = Vec::with_capacity(self.data.peers.len()); @@ -330,45 +343,8 @@ impl CreateFlowProcedure { .invalidate(&ctx, &caches) .await?; - if did_replace { - self.on_flownode_drop_previous_flows().await?; - } - Ok(Status::done_with_output(flow_id)) } - - async fn on_flownode_drop_previous_flows(&self) -> Result<()> { - let Some(prev_flow_info) = self.data.prev_flow_info_value.as_ref() else { - return Ok(()); - }; - if prev_flow_info.flownode_ids().is_empty() { - return Ok(()); - } - - let flow_id = self.data.flow_id.unwrap(); - let mut drop_flow_tasks = Vec::with_capacity(self.data.prev_peers.len()); - for peer in &self.data.prev_peers { - let requester = self.context.node_manager.flownode(peer).await; - let request = FlowRequest { - body: Some(flow_request::Body::Drop(DropRequest { - flow_id: Some(api::v1::FlowId { id: flow_id }), - })), - ..Default::default() - }; - drop_flow_tasks.push(async move { - requester - .handle(request) - .await - .map_err(add_peer_context_if_needed(peer.clone())) - }); - } - join_all(drop_flow_tasks) - .await - .into_iter() - .collect::>>()?; - - Ok(()) - } } #[async_trait] @@ -519,15 +495,11 @@ pub struct CreateFlowData { /// For verify if prev value is consistent when need to update flow metadata. /// only set when `or_replace` is true. pub(crate) prev_flow_info_value: Option>, - #[serde(default)] - pub(crate) prev_peers: Vec, /// Only set to true when replace actually happened. /// This is used to determine whether to invalidate the cache. #[serde(default)] pub(crate) did_replace: bool, pub(crate) flow_type: Option, - #[serde(default)] - pub(crate) last_activation_error: Option, } impl CreateFlowData { @@ -629,7 +601,6 @@ impl From<&CreateFlowData> for (FlowInfoValue, Vec<(FlowPartitionId, FlowRouteVa } else { FlowStatus::PendingSources }, - last_activation_error: value.last_activation_error.clone(), created_time: create_time, updated_time: chrono::Utc::now(), }; diff --git a/src/common/meta/src/ddl/drop_flow/metadata.rs b/src/common/meta/src/ddl/drop_flow/metadata.rs index 0437098be358..7afd00f9d5ac 100644 --- a/src/common/meta/src/ddl/drop_flow/metadata.rs +++ b/src/common/meta/src/ddl/drop_flow/metadata.rs @@ -43,7 +43,7 @@ impl DropFlowProcedure { .map(|(_, value)| value) .collect::>(); ensure!( - !flow_route_values.is_empty(), + flow_info_value.is_pending() || !flow_route_values.is_empty(), error::FlowRouteNotFoundSnafu { flow_name: format_full_flow_name(catalog_name, flow_name), } diff --git a/src/common/meta/src/ddl/flow_meta.rs b/src/common/meta/src/ddl/flow_meta.rs index fd41690b260e..85c1f3e989cc 100644 --- a/src/common/meta/src/ddl/flow_meta.rs +++ b/src/common/meta/src/ddl/flow_meta.rs @@ -59,13 +59,8 @@ impl FlowMetadataAllocator { /// Allocates the [FlowId] and [Peer]s. pub async fn create(&self, partitions: usize) -> Result<(FlowId, Vec)> { let flow_id = self.allocate_flow_id().await?; - let peers = self.alloc_peers(partitions).await?; + let peers = self.peer_allocator.alloc(partitions).await?; Ok((flow_id, peers)) } - - pub async fn alloc_peers(&self, partitions: usize) -> Result> { - let peers = self.peer_allocator.alloc(partitions).await?; - Ok(peers) - } } diff --git a/src/common/meta/src/ddl/tests.rs b/src/common/meta/src/ddl/tests.rs index a09283205bf2..0700259cf844 100644 --- a/src/common/meta/src/ddl/tests.rs +++ b/src/common/meta/src/ddl/tests.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod activate_flow; mod alter_logical_tables; mod alter_table; mod create_flow; diff --git a/src/common/meta/src/ddl/tests/activate_flow.rs b/src/common/meta/src/ddl/tests/activate_flow.rs deleted file mode 100644 index 9502b5dcb020..000000000000 --- a/src/common/meta/src/ddl/tests/activate_flow.rs +++ /dev/null @@ -1,329 +0,0 @@ -// Copyright 2023 Greptime Team -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::{Arc, Mutex}; - -use api::v1::flow::flow_request::Body as PbFlowRequest; -use api::v1::flow::{CreateRequest, FlowRequest, FlowResponse}; -use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; -use common_procedure_test::execute_procedure_until_done; -use common_time::TimeToLive; -use table::table_name::TableName; - -use crate::ddl::activate_flow::ActivatePendingFlowProcedure; -use crate::ddl::create_flow::{DEFER_ON_MISSING_SOURCE_KEY, FlowType}; -use crate::ddl::test_util::create_table::test_create_table_task; -use crate::ddl::tests::create_flow::create_test_pending_flow; -use crate::key::table_route::TableRouteValue; -use crate::test_util::{MockFlownodeHandler, MockFlownodeManager, new_ddl_context}; - -#[derive(Clone, Default)] -struct RecordingFlownodeHandler { - create_requests: Arc>>, -} - -#[async_trait::async_trait] -impl MockFlownodeHandler for RecordingFlownodeHandler { - async fn handle( - &self, - _peer: &crate::peer::Peer, - request: FlowRequest, - ) -> crate::error::Result { - if let Some(PbFlowRequest::Create(create_req)) = request.body { - self.create_requests.lock().unwrap().push(create_req); - } - - Ok(FlowResponse { - affected_rows: 0, - ..Default::default() - }) - } - - async fn handle_inserts( - &self, - _peer: &crate::peer::Peer, - _requests: api::v1::region::InsertRequests, - ) -> crate::error::Result { - unreachable!() - } -} - -#[tokio::test] -async fn test_activate_pending_flow() { - let source_table_names = vec![TableName::new( - DEFAULT_CATALOG_NAME, - DEFAULT_SCHEMA_NAME, - "activate_source_table", - )]; - let sink_table_name = TableName::new( - DEFAULT_CATALOG_NAME, - DEFAULT_SCHEMA_NAME, - "activate_sink_table", - ); - let node_manager = Arc::new(MockFlownodeManager::new( - crate::ddl::test_util::flownode_handler::NaiveFlownodeHandler, - )); - let ddl_context = new_ddl_context(node_manager); - - let flow_id = create_test_pending_flow( - &ddl_context, - "activate_pending_flow", - source_table_names.clone(), - sink_table_name, - ) - .await; - - let pending_flow = ddl_context - .flow_metadata_manager - .flow_info_manager() - .get(flow_id) - .await - .unwrap() - .unwrap(); - assert!(pending_flow.is_pending()); - - let create_table_task = test_create_table_task("activate_source_table", 1024); - ddl_context - .table_metadata_manager - .create_table_metadata( - create_table_task.table_info.clone(), - TableRouteValue::physical(vec![]), - Default::default(), - ) - .await - .unwrap(); - - let mut procedure = ActivatePendingFlowProcedure::new( - flow_id, - DEFAULT_CATALOG_NAME.to_string(), - ddl_context.clone(), - ); - let output = execute_procedure_until_done(&mut procedure).await.unwrap(); - let activated_flow_id = output.downcast_ref::().unwrap(); - assert_eq!(*activated_flow_id, flow_id); - - let activated_flow = ddl_context - .flow_metadata_manager - .flow_info_manager() - .get(flow_id) - .await - .unwrap() - .unwrap(); - assert!(activated_flow.is_active()); - assert_eq!(activated_flow.unresolved_source_table_names().len(), 0); - assert_eq!(activated_flow.source_table_ids(), &[1024]); - assert_eq!(activated_flow.last_activation_error(), &None); - assert!(!activated_flow.flownode_ids().is_empty()); -} - -#[tokio::test] -async fn test_activate_pending_batching_flow_with_instant_ttl_source() { - let source_table_names = vec![TableName::new( - DEFAULT_CATALOG_NAME, - DEFAULT_SCHEMA_NAME, - "instant_ttl_source_table", - )]; - let sink_table_name = TableName::new( - DEFAULT_CATALOG_NAME, - DEFAULT_SCHEMA_NAME, - "instant_ttl_sink_table", - ); - let node_manager = Arc::new(MockFlownodeManager::new( - crate::ddl::test_util::flownode_handler::NaiveFlownodeHandler, - )); - let ddl_context = new_ddl_context(node_manager); - - let flow_id = create_test_pending_flow( - &ddl_context, - "instant_ttl_pending_flow", - source_table_names.clone(), - sink_table_name, - ) - .await; - - let mut create_table_task = test_create_table_task("instant_ttl_source_table", 1025); - create_table_task.table_info.meta.options.ttl = Some(TimeToLive::Instant); - ddl_context - .table_metadata_manager - .create_table_metadata( - create_table_task.table_info.clone(), - TableRouteValue::physical(vec![]), - Default::default(), - ) - .await - .unwrap(); - - let mut procedure = ActivatePendingFlowProcedure::new( - flow_id, - DEFAULT_CATALOG_NAME.to_string(), - ddl_context.clone(), - ); - let output = execute_procedure_until_done(&mut procedure).await.unwrap(); - let activated_flow_id = output.downcast_ref::().unwrap(); - assert_eq!(*activated_flow_id, flow_id); - - let activated_flow = ddl_context - .flow_metadata_manager - .flow_info_manager() - .get(flow_id) - .await - .unwrap() - .unwrap(); - assert!(activated_flow.is_active()); - assert_eq!(activated_flow.unresolved_source_table_names().len(), 0); - assert_eq!(activated_flow.source_table_ids(), &[1025]); - assert_eq!(activated_flow.last_activation_error(), &None); -} - -#[tokio::test] -async fn test_activate_pending_flow_uses_replace_semantics() { - let source_table_names = vec![TableName::new( - DEFAULT_CATALOG_NAME, - DEFAULT_SCHEMA_NAME, - "activate_replace_source_table", - )]; - let sink_table_name = TableName::new( - DEFAULT_CATALOG_NAME, - DEFAULT_SCHEMA_NAME, - "activate_replace_sink_table", - ); - - let handler = RecordingFlownodeHandler::default(); - let node_manager = Arc::new(MockFlownodeManager::new(handler.clone())); - let ddl_context = new_ddl_context(node_manager); - - let flow_id = create_test_pending_flow( - &ddl_context, - "activate_pending_flow_replace_semantics", - source_table_names, - sink_table_name, - ) - .await; - - let create_table_task = test_create_table_task("activate_replace_source_table", 1027); - ddl_context - .table_metadata_manager - .create_table_metadata( - create_table_task.table_info.clone(), - TableRouteValue::physical(vec![]), - Default::default(), - ) - .await - .unwrap(); - - let mut procedure = - ActivatePendingFlowProcedure::new(flow_id, DEFAULT_CATALOG_NAME.to_string(), ddl_context); - let output = execute_procedure_until_done(&mut procedure).await.unwrap(); - let activated_flow_id = output.downcast_ref::().unwrap(); - assert_eq!(*activated_flow_id, flow_id); - - let create_requests = handler.create_requests.lock().unwrap(); - assert!(!create_requests.is_empty()); - for req in create_requests.iter() { - assert!(!req.create_if_not_exists); - assert!(req.or_replace); - } -} - -#[tokio::test] -async fn test_activate_pending_flow_preserves_streaming_type() { - let source_table_names = vec![TableName::new( - DEFAULT_CATALOG_NAME, - DEFAULT_SCHEMA_NAME, - "activate_streaming_source_table", - )]; - let sink_table_name = TableName::new( - DEFAULT_CATALOG_NAME, - DEFAULT_SCHEMA_NAME, - "activate_streaming_sink_table", - ); - - let handler = RecordingFlownodeHandler::default(); - let node_manager = Arc::new(MockFlownodeManager::new(handler.clone())); - let ddl_context = new_ddl_context(node_manager); - - let flow_id = create_test_pending_flow( - &ddl_context, - "activate_streaming_pending_flow", - source_table_names, - sink_table_name, - ) - .await; - - let pending_flow_info = ddl_context - .flow_metadata_manager - .flow_info_manager() - .get_raw(flow_id) - .await - .unwrap() - .unwrap(); - let mut updated_flow_info = pending_flow_info.get_inner_ref().clone(); - updated_flow_info.options.insert( - FlowType::FLOW_TYPE_KEY.to_string(), - FlowType::Streaming.to_string(), - ); - ddl_context - .flow_metadata_manager - .update_flow_metadata(flow_id, &pending_flow_info, &updated_flow_info, vec![]) - .await - .unwrap(); - - let create_table_task = test_create_table_task("activate_streaming_source_table", 1028); - ddl_context - .table_metadata_manager - .create_table_metadata( - create_table_task.table_info.clone(), - TableRouteValue::physical(vec![]), - Default::default(), - ) - .await - .unwrap(); - - let mut procedure = ActivatePendingFlowProcedure::new( - flow_id, - DEFAULT_CATALOG_NAME.to_string(), - ddl_context.clone(), - ); - let output = execute_procedure_until_done(&mut procedure).await.unwrap(); - let activated_flow_id = output.downcast_ref::().unwrap(); - assert_eq!(*activated_flow_id, flow_id); - - let activated_flow = ddl_context - .flow_metadata_manager - .flow_info_manager() - .get(flow_id) - .await - .unwrap() - .unwrap(); - assert!(activated_flow.is_active()); - assert_eq!( - activated_flow - .options() - .get(FlowType::FLOW_TYPE_KEY) - .map(String::as_str), - Some(FlowType::STREAMING) - ); - - let create_requests = handler.create_requests.lock().unwrap(); - assert!(!create_requests.is_empty()); - for req in create_requests.iter() { - assert!(!req.flow_options.contains_key(DEFER_ON_MISSING_SOURCE_KEY)); - assert_eq!( - req.flow_options - .get(FlowType::FLOW_TYPE_KEY) - .map(String::as_str), - Some(FlowType::STREAMING) - ); - } -} diff --git a/src/common/meta/src/ddl/tests/create_flow.rs b/src/common/meta/src/ddl/tests/create_flow.rs index fd704c6bfaf0..a04d5b692a23 100644 --- a/src/common/meta/src/ddl/tests/create_flow.rs +++ b/src/common/meta/src/ddl/tests/create_flow.rs @@ -14,10 +14,9 @@ use std::assert_matches; use std::collections::HashMap; -use std::sync::{Arc, Mutex}; +use std::sync::Arc; -use api::v1::flow::flow_request::Body as PbFlowRequest; -use api::v1::flow::{CreateRequest, DropRequest, FlowRequest, FlowResponse}; +use api::v1::flow::CreateRequest; use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; use common_procedure::Status; use common_procedure_test::execute_procedure_until_done; @@ -36,37 +35,6 @@ use crate::key::table_route::TableRouteValue; use crate::rpc::ddl::{CreateFlowTask, FlowQueryContext, QueryContext}; use crate::test_util::{MockFlownodeManager, new_ddl_context}; -#[derive(Clone, Default)] -struct RecordingDropFlownodeHandler { - drop_requests: Arc>>, -} - -#[async_trait::async_trait] -impl crate::test_util::MockFlownodeHandler for RecordingDropFlownodeHandler { - async fn handle( - &self, - _peer: &crate::peer::Peer, - request: FlowRequest, - ) -> crate::error::Result { - if let Some(PbFlowRequest::Drop(drop_req)) = request.body { - self.drop_requests.lock().unwrap().push(drop_req); - } - - Ok(FlowResponse { - affected_rows: 0, - ..Default::default() - }) - } - - async fn handle_inserts( - &self, - _peer: &crate::peer::Peer, - _requests: api::v1::region::InsertRequests, - ) -> crate::error::Result { - unreachable!() - } -} - fn test_query_context() -> QueryContext { QueryContext { current_catalog: DEFAULT_CATALOG_NAME.to_string(), @@ -239,10 +207,8 @@ fn test_create_request_strips_defer_on_missing_source_runtime_option() { sst_min_sequences: HashMap::new(), }, prev_flow_info_value: None, - prev_peers: vec![], did_replace: false, flow_type: Some(FlowType::Batching), - last_activation_error: None, }; let request: CreateRequest = (&data).into(); @@ -356,7 +322,7 @@ async fn test_create_flow() { } #[tokio::test] -async fn test_replace_pending_flow_activates_with_allocated_peers() { +async fn test_replace_pending_flow_with_active_flow_is_unsupported() { let source_table_name = TableName::new( DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, @@ -408,32 +374,16 @@ async fn test_replace_pending_flow_activates_with_allocated_peers() { replace_task.or_replace = true; let query_ctx = test_query_context(); let mut procedure = CreateFlowProcedure::new(replace_task, query_ctx, ddl_context.clone()); - let output = execute_procedure_until_done(&mut procedure).await.unwrap(); - let flow_id = *output.downcast_ref::().unwrap(); - assert_eq!(flow_id, pending_flow_id); - - let replaced_flow = ddl_context - .flow_metadata_manager - .flow_info_manager() - .get(flow_id) - .await - .unwrap() - .unwrap(); - assert!(replaced_flow.is_active()); - assert_eq!(replaced_flow.source_table_ids(), &[1026]); - assert!(!replaced_flow.flownode_ids().is_empty()); - - let routes = ddl_context - .flow_metadata_manager - .flow_route_manager() - .routes(flow_id) - .await - .unwrap(); - assert!(!routes.is_empty()); + let err = procedure.on_prepare().await.unwrap_err(); + assert_matches!(err, error::Error::Unsupported { .. }); + assert!( + err.to_string() + .contains("Replacing between pending and active flow states") + ); } #[tokio::test] -async fn test_replace_active_flow_to_pending_drops_old_flows() { +async fn test_replace_active_flow_with_pending_flow_is_unsupported() { let existing_source_table = TableName::new( DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, @@ -450,8 +400,7 @@ async fn test_replace_active_flow_to_pending_drops_old_flows() { "replace_active_sink_table", ); - let handler = RecordingDropFlownodeHandler::default(); - let node_manager = Arc::new(MockFlownodeManager::new(handler.clone())); + let node_manager = Arc::new(MockFlownodeManager::new(NaiveFlownodeHandler)); let ddl_context = new_ddl_context(node_manager); let create_table_task = test_create_table_task("replace_active_source_table", 2026); @@ -465,7 +414,7 @@ async fn test_replace_active_flow_to_pending_drops_old_flows() { .await .unwrap(); - let flow_id = create_test_flow( + let _flow_id = create_test_flow( &ddl_context, "replace_active_flow_to_pending", vec![existing_source_table], @@ -483,29 +432,12 @@ async fn test_replace_active_flow_to_pending_drops_old_flows() { replace_task.or_replace = true; let query_ctx = test_query_context(); let mut procedure = CreateFlowProcedure::new(replace_task, query_ctx, ddl_context.clone()); - let output = execute_procedure_until_done(&mut procedure).await.unwrap(); - let replaced_flow_id = *output.downcast_ref::().unwrap(); - assert_eq!(replaced_flow_id, flow_id); - - let replaced_flow = ddl_context - .flow_metadata_manager - .flow_info_manager() - .get(flow_id) - .await - .unwrap() - .unwrap(); - assert!(replaced_flow.is_pending()); - assert_eq!( - replaced_flow - .options() - .get(DEFER_ON_MISSING_SOURCE_KEY) - .map(String::as_str), - Some("true") + let err = procedure.on_prepare().await.unwrap_err(); + assert_matches!(err, error::Error::Unsupported { .. }); + assert!( + err.to_string() + .contains("Replacing between pending and active flow states") ); - - let drop_requests = handler.drop_requests.lock().unwrap(); - assert!(!drop_requests.is_empty()); - assert_eq!(drop_requests[0].flow_id.as_ref().unwrap().id, flow_id); } #[tokio::test] @@ -623,10 +555,8 @@ fn test_create_flow_data_new_format_serialization() { unresolved_source_table_names: vec![], flow_context, prev_flow_info_value: None, - prev_peers: vec![], did_replace: false, flow_type: None, - last_activation_error: None, }; let serialized = serde_json::to_string(&data).unwrap(); @@ -688,10 +618,8 @@ fn test_flow_info_conversion_with_flow_context() { unresolved_source_table_names: vec![], flow_context, prev_flow_info_value: None, - prev_peers: vec![], did_replace: false, flow_type: Some(FlowType::Batching), - last_activation_error: None, }; let (flow_info, _routes) = (&data).into(); diff --git a/src/common/meta/src/ddl/tests/drop_flow.rs b/src/common/meta/src/ddl/tests/drop_flow.rs index af34da4809d3..400fd2e118bf 100644 --- a/src/common/meta/src/ddl/tests/drop_flow.rs +++ b/src/common/meta/src/ddl/tests/drop_flow.rs @@ -23,7 +23,7 @@ use table::table_name::TableName; use crate::ddl::drop_flow::DropFlowProcedure; use crate::ddl::test_util::create_table::test_create_table_task; use crate::ddl::test_util::flownode_handler::NaiveFlownodeHandler; -use crate::ddl::tests::create_flow::create_test_flow; +use crate::ddl::tests::create_flow::{create_test_flow, create_test_pending_flow}; use crate::error; use crate::key::table_route::TableRouteValue; use crate::rpc::ddl::DropFlowTask; @@ -91,3 +91,45 @@ async fn test_drop_flow() { let err = procedure.on_prepare().await.unwrap_err(); assert_matches!(err, error::Error::FlowNotFound { .. }); } + +#[tokio::test] +async fn test_drop_pending_flow_without_routes() { + let source_table_name = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "drop_pending_missing_source_table", + ); + let sink_table_name = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "drop_pending_sink_table", + ); + let node_manager = Arc::new(MockFlownodeManager::new(NaiveFlownodeHandler)); + let ddl_context = new_ddl_context(node_manager); + + let flow_id = create_test_pending_flow( + &ddl_context, + "drop_pending_flow", + vec![source_table_name], + sink_table_name, + ) + .await; + let flow_info = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(flow_id) + .await + .unwrap() + .unwrap(); + assert!(flow_info.is_pending()); + assert!(flow_info.flownode_ids().is_empty()); + + let task = test_drop_flow_task("drop_pending_flow", flow_id, false); + let mut procedure = DropFlowProcedure::new(task, ddl_context.clone()); + execute_procedure_until_done(&mut procedure).await; + + let task = test_drop_flow_task("drop_pending_flow", flow_id, false); + let mut procedure = DropFlowProcedure::new(task, ddl_context); + let err = procedure.on_prepare().await.unwrap_err(); + assert_matches!(err, error::Error::FlowNotFound { .. }); +} diff --git a/src/common/meta/src/ddl_manager.rs b/src/common/meta/src/ddl_manager.rs index 2f5dc928fc5c..d0619ca74fbd 100644 --- a/src/common/meta/src/ddl_manager.rs +++ b/src/common/meta/src/ddl_manager.rs @@ -29,7 +29,6 @@ use snafu::{OptionExt, ResultExt, ensure}; use store_api::storage::TableId; use table::table_name::TableName; -use crate::ddl::activate_flow::ActivatePendingFlowProcedure; use crate::ddl::alter_database::AlterDatabaseProcedure; use crate::ddl::alter_logical_tables::AlterLogicalTablesProcedure; use crate::ddl::alter_table::AlterTableProcedure; @@ -233,7 +232,6 @@ impl DdlManager { CreateLogicalTablesProcedure, CreateViewProcedure, CreateFlowProcedure, - ActivatePendingFlowProcedure, AlterTableProcedure, AlterLogicalTablesProcedure, AlterDatabaseProcedure, @@ -490,20 +488,6 @@ impl DdlManager { self.execute_procedure_and_wait(procedure_with_id).await } - /// Submits and executes a pending-flow activation task. - #[tracing::instrument(skip_all)] - pub async fn submit_activate_pending_flow_task( - &self, - flow_id: crate::key::FlowId, - catalog_name: String, - ) -> Result<(ProcedureId, Option)> { - let context = self.create_context(); - let procedure = ActivatePendingFlowProcedure::new(flow_id, catalog_name, context); - let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure)); - - self.execute_procedure_and_wait(procedure_with_id).await - } - /// Submits and executes a drop flow task. #[tracing::instrument(skip_all)] pub async fn submit_drop_flow_task( diff --git a/src/common/meta/src/key/flow.rs b/src/common/meta/src/key/flow.rs index e1223fd67f38..bc9aaaa6b370 100644 --- a/src/common/meta/src/key/flow.rs +++ b/src/common/meta/src/key/flow.rs @@ -533,7 +533,6 @@ mod tests { comment: "hi".to_string(), options: Default::default(), status: FlowStatus::Active, - last_activation_error: None, created_time: chrono::Utc::now(), updated_time: chrono::Utc::now(), } @@ -789,7 +788,6 @@ mod tests { comment: "hi".to_string(), options: Default::default(), status: FlowStatus::Active, - last_activation_error: None, created_time: chrono::Utc::now(), updated_time: chrono::Utc::now(), }; @@ -1170,7 +1168,6 @@ mod tests { comment: "hi".to_string(), options: Default::default(), status: FlowStatus::Active, - last_activation_error: None, created_time: chrono::Utc::now(), updated_time: chrono::Utc::now(), }; diff --git a/src/common/meta/src/key/flow/flow_info.rs b/src/common/meta/src/key/flow/flow_info.rs index 0957d5681576..522860e418c6 100644 --- a/src/common/meta/src/key/flow/flow_info.rs +++ b/src/common/meta/src/key/flow/flow_info.rs @@ -166,8 +166,6 @@ pub struct FlowInfoValue { pub options: HashMap, #[serde(default)] pub status: FlowStatus, - #[serde(default)] - pub last_activation_error: Option, /// The created time #[serde(default)] pub created_time: DateTime, @@ -252,10 +250,6 @@ impl FlowInfoValue { &self.status } - pub fn last_activation_error(&self) -> &Option { - &self.last_activation_error - } - pub fn created_time(&self) -> &DateTime { &self.created_time } diff --git a/src/meta-srv/src/error.rs b/src/meta-srv/src/error.rs index dcb77f90f38f..a0f800f98164 100644 --- a/src/meta-srv/src/error.rs +++ b/src/meta-srv/src/error.rs @@ -135,20 +135,6 @@ pub enum Error { source: common_meta::error::Error, }, - #[snafu(display("Failed to init pending flow reconcile manager"))] - InitPendingFlowReconcileManager { - #[snafu(implicit)] - location: Location, - source: BoxedError, - }, - - #[snafu(display("Failed to reconcile pending flows"))] - PendingFlowReconcile { - #[snafu(implicit)] - location: Location, - source: common_meta::error::Error, - }, - #[snafu(display("Failed to create default catalog and schema"))] InitMetadata { #[snafu(implicit)] @@ -1291,8 +1277,6 @@ impl ErrorExt for Error { Error::InitMetadata { source, .. } | Error::InitDdlManager { source, .. } | Error::InitReconciliationManager { source, .. } => source.status_code(), - Error::InitPendingFlowReconcileManager { source, .. } => source.status_code(), - Error::PendingFlowReconcile { source, .. } => source.status_code(), Error::BuildTlsOptions { source, .. } => source.status_code(), Error::Other { source, .. } => source.status_code(), diff --git a/src/meta-srv/src/flow.rs b/src/meta-srv/src/flow.rs deleted file mode 100644 index 8f7c22b9b1ab..000000000000 --- a/src/meta-srv/src/flow.rs +++ /dev/null @@ -1,164 +0,0 @@ -// Copyright 2023 Greptime Team -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::sync::Arc; -use std::time::Duration; - -use common_meta::ddl_manager::DdlManagerRef; -use common_meta::key::table_name::TableNameKey; -use common_telemetry::{error, info}; -use futures::TryStreamExt; -use snafu::ResultExt; -use tokio::sync::mpsc::{Receiver, Sender}; - -use crate::define_ticker; -use crate::error::{PendingFlowReconcileSnafu, Result}; - -const PENDING_FLOW_RECONCILE_INTERVAL: Duration = Duration::from_secs(10); - -pub enum Event { - Tick, -} - -pub type PendingFlowReconcileTickerRef = Arc; - -define_ticker!( - PendingFlowReconcileTicker, - event_type = Event, - event_value = Event::Tick -); - -pub struct PendingFlowReconcileManager { - ddl_manager: DdlManagerRef, - receiver: Receiver, -} - -impl PendingFlowReconcileManager { - pub fn new(ddl_manager: DdlManagerRef) -> (Self, PendingFlowReconcileTicker) { - let (sender, receiver) = Self::channel(); - ( - Self { - ddl_manager, - receiver, - }, - PendingFlowReconcileTicker::new(PENDING_FLOW_RECONCILE_INTERVAL, sender), - ) - } - - fn channel() -> (Sender, Receiver) { - tokio::sync::mpsc::channel(8) - } - - pub fn try_start(mut self) -> Result<()> { - common_runtime::spawn_global(async move { self.run().await }); - info!("Pending flow reconcile manager started"); - Ok(()) - } - - async fn run(&mut self) { - while let Some(event) = self.receiver.recv().await { - match event { - Event::Tick => { - if let Err(e) = self.handle_tick().await { - error!(e; "Failed to reconcile pending flows"); - } - } - } - } - } - - async fn handle_tick(&self) -> Result<()> { - let ddl_context = self.ddl_manager.create_context(); - let ddl_manager = self.ddl_manager.clone(); - ddl_context - .flow_metadata_manager - .flow_info_manager() - .flow_infos() - .try_for_each(move |(flow_id, flow_info)| { - let ddl_context = ddl_context.clone(); - let ddl_manager = ddl_manager.clone(); - async move { - if !flow_info.is_pending() { - return Ok(()); - } - - let current_flow_info = ddl_context - .flow_metadata_manager - .flow_info_manager() - .get_raw(flow_id) - .await; - let current_flow_info = match current_flow_info { - Ok(current_flow_info) => current_flow_info, - Err(e) => { - error!(e; "Failed to load flow metadata for pending flow {}", flow_id); - return Ok(()); - } - }; - let Some(current_flow_info) = current_flow_info else { - return Ok(()); - }; - if !current_flow_info.get_inner_ref().is_pending() { - return Ok(()); - } - - let unresolved_source_table_names = current_flow_info - .get_inner_ref() - .unresolved_source_table_names(); - if !unresolved_source_table_names.is_empty() { - let unresolved_table_keys = unresolved_source_table_names - .iter() - .map(|name| { - TableNameKey::new( - &name.catalog_name, - &name.schema_name, - &name.table_name, - ) - }) - .collect::>(); - let resolved_tables = ddl_context - .table_metadata_manager - .table_name_manager() - .batch_get(unresolved_table_keys) - .await; - let resolved_tables = match resolved_tables { - Ok(resolved_tables) => resolved_tables, - Err(e) => { - error!(e; "Failed to resolve source tables for pending flow {}", flow_id); - return Ok(()); - } - }; - if resolved_tables.iter().all(|table_id| table_id.is_none()) { - return Ok(()); - } - } - - if let Err(e) = ddl_manager - .submit_activate_pending_flow_task( - flow_id, - current_flow_info.get_inner_ref().catalog_name().clone(), - ) - .await - { - error!(e; "Failed to reconcile pending flow {}", flow_id); - } - - Ok(()) - } - }) - .await - .context(PendingFlowReconcileSnafu)?; - - Ok(()) - } -} diff --git a/src/meta-srv/src/lib.rs b/src/meta-srv/src/lib.rs index c9045c4df98c..70ce449bba80 100644 --- a/src/meta-srv/src/lib.rs +++ b/src/meta-srv/src/lib.rs @@ -22,7 +22,6 @@ pub mod discovery; pub mod error; pub mod events; mod failure_detector; -pub mod flow; pub mod gc; pub mod handler; pub mod key; diff --git a/src/meta-srv/src/metasrv.rs b/src/meta-srv/src/metasrv.rs index 6acb1b8076ed..654457814888 100644 --- a/src/meta-srv/src/metasrv.rs +++ b/src/meta-srv/src/metasrv.rs @@ -71,7 +71,6 @@ use crate::error::{ StartTelemetryTaskSnafu, StopProcedureManagerSnafu, }; use crate::failure_detector::PhiAccrualFailureDetectorOptions; -use crate::flow::PendingFlowReconcileTickerRef; use crate::gc::{GcSchedulerOptions, GcTickerRef}; use crate::handler::{HeartbeatHandlerGroupBuilder, HeartbeatHandlerGroupRef}; use crate::procedure::ProcedureManagerListenerAdapter; @@ -570,7 +569,6 @@ pub struct Metasrv { region_flush_ticker: Option, table_id_allocator: ResourceIdAllocatorRef, reconciliation_manager: ReconciliationManagerRef, - pending_flow_reconcile_ticker: Option, resource_stat: ResourceStatRef, gc_ticker: Option, database_operator: DatabaseOperatorRef, @@ -637,9 +635,6 @@ impl Metasrv { if let Some(gc_ticker) = &self.gc_ticker { leadership_change_notifier.add_listener(gc_ticker.clone() as _); } - if let Some(pending_flow_reconcile_ticker) = &self.pending_flow_reconcile_ticker { - leadership_change_notifier.add_listener(pending_flow_reconcile_ticker.clone() as _); - } if let Some(customizer) = self.plugins.get::() { customizer.customize(&mut leadership_change_notifier); } diff --git a/src/meta-srv/src/metasrv/builder.rs b/src/meta-srv/src/metasrv/builder.rs index d30775f1a49f..54a94fffbf52 100644 --- a/src/meta-srv/src/metasrv/builder.rs +++ b/src/meta-srv/src/metasrv/builder.rs @@ -56,7 +56,6 @@ use crate::cache_invalidator::MetasrvCacheInvalidator; use crate::cluster::MetaPeerClientRef; use crate::error::{self, BuildWalProviderSnafu, OtherSnafu, Result}; use crate::events::EventHandlerImpl; -use crate::flow::{PendingFlowReconcileManager, PendingFlowReconcileTickerRef}; use crate::gc::GcScheduler; use crate::greptimedb_telemetry::get_greptimedb_telemetry_task; use crate::handler::failure_handler::RegionFailureHandler; @@ -559,15 +558,6 @@ impl MetasrvBuilder { .try_start() .context(error::InitReconciliationManagerSnafu)?; - let (pending_flow_reconcile_manager, pending_flow_reconcile_ticker) = - PendingFlowReconcileManager::new(ddl_manager.clone()); - pending_flow_reconcile_manager - .try_start() - .map_err(common_error::ext::BoxedError::new) - .context(error::InitPendingFlowReconcileManagerSnafu)?; - let pending_flow_reconcile_ticker: Option = - Some(Arc::new(pending_flow_reconcile_ticker)); - let mut resource_stat = ResourceStatImpl::default(); resource_stat.start_collect_cpu_usage(); @@ -609,7 +599,6 @@ impl MetasrvBuilder { region_flush_ticker, table_id_allocator, reconciliation_manager, - pending_flow_reconcile_ticker, topic_stats_registry, resource_stat: Arc::new(resource_stat), gc_ticker, diff --git a/tests/cases/standalone/common/flow/flow_pending.result b/tests/cases/standalone/common/flow/flow_pending.result index 80a56cb22b44..d6fe01b38a8f 100644 --- a/tests/cases/standalone/common/flow/flow_pending.result +++ b/tests/cases/standalone/common/flow/flow_pending.result @@ -41,108 +41,12 @@ WHERE flow_name = 'pending_with_defer'; | AS SELECT val FROM pending_source WHERE val > 10 | | | | | | +--------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ -CREATE TABLE pending_source ( - val INT, - ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, - TIME INDEX(ts) -); - -Affected Rows: 0 - --- SQLNESS SLEEP 12s -SELECT - flow_definition, - source_table_ids, - source_table_names, - flownode_ids, - options LIKE '%"defer_on_missing_source":"true"%' AS has_defer_option, - options LIKE '%"flow_type":"batching"%' AS has_flow_type_option -FROM INFORMATION_SCHEMA.FLOWS -WHERE flow_name = 'pending_with_defer'; - -+--------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ -| flow_definition | source_table_ids | source_table_names | flownode_ids | has_defer_option | has_flow_type_option | -+--------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ -| CREATE FLOW IF NOT EXISTS pending_with_defer | [] | | {} | true | true | -| SINK TO public.pending_sink | | | | | | -| WITH (defer_on_missing_source = 'true') | | | | | | -| AS SELECT val FROM pending_source WHERE val > 10 | | | | | | -+--------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ - -INSERT INTO pending_source VALUES (10, 0), (11, 1), (12, 2); - -Affected Rows: 3 - --- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | -ADMIN FLUSH_FLOW('pending_with_defer'); - -Error: 1002(Unexpected), Failed to execute admin function flush_flow: Execution error: Can't found alive flownode - -SELECT val FROM pending_sink ORDER BY val; - -Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.public.pending_sink - -CREATE OR REPLACE FLOW pending_with_defer -SINK TO pending_sink -WITH (defer_on_missing_source = true) -AS SELECT val FROM pending_replacement_source WHERE val > 100; - -Affected Rows: 0 - -SELECT - flow_definition, - source_table_ids, - source_table_names, - flownode_ids, - options LIKE '%"defer_on_missing_source":"true"%' AS has_defer_option, - options LIKE '%"flow_type":"batching"%' AS has_flow_type_option -FROM INFORMATION_SCHEMA.FLOWS -WHERE flow_name = 'pending_with_defer'; - -+---------------------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ -| flow_definition | source_table_ids | source_table_names | flownode_ids | has_defer_option | has_flow_type_option | -+---------------------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ -| CREATE FLOW IF NOT EXISTS pending_with_defer | [] | | {} | true | true | -| SINK TO public.pending_sink | | | | | | -| WITH (defer_on_missing_source = 'true') | | | | | | -| AS SELECT val FROM pending_replacement_source WHERE val > 100 | | | | | | -+---------------------------------------------------------------+------------------+--------------------+--------------+------------------+----------------------+ - -CREATE TABLE pending_replacement_source ( - val INT, - ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, - TIME INDEX(ts) -); - -Affected Rows: 0 - --- SQLNESS SLEEP 12s -INSERT INTO pending_replacement_source VALUES (99, 3), (101, 4), (102, 5); - -Affected Rows: 3 - --- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | -ADMIN FLUSH_FLOW('pending_with_defer'); - -Error: 1002(Unexpected), Failed to execute admin function flush_flow: Execution error: Can't found alive flownode - -SELECT val FROM pending_sink ORDER BY val; - -Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.public.pending_sink - DROP FLOW pending_with_defer; -Error: 1002(Unexpected), Flow route not found: 'greptime.pending_with_defer' - -DROP TABLE pending_sink; - -Error: 4001(TableNotFound), Table not found: greptime.public.pending_sink - -DROP TABLE pending_source; - Affected Rows: 0 -DROP TABLE pending_replacement_source; +SELECT flow_name FROM INFORMATION_SCHEMA.FLOWS WHERE flow_name = 'pending_with_defer'; -Affected Rows: 0 +++ +++ diff --git a/tests/cases/standalone/common/flow/flow_pending.sql b/tests/cases/standalone/common/flow/flow_pending.sql index d74de54446fa..498f5b278250 100644 --- a/tests/cases/standalone/common/flow/flow_pending.sql +++ b/tests/cases/standalone/common/flow/flow_pending.sql @@ -19,65 +19,6 @@ SELECT FROM INFORMATION_SCHEMA.FLOWS WHERE flow_name = 'pending_with_defer'; -CREATE TABLE pending_source ( - val INT, - ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, - TIME INDEX(ts) -); - --- SQLNESS SLEEP 12s - -SELECT - flow_definition, - source_table_ids, - source_table_names, - flownode_ids, - options LIKE '%"defer_on_missing_source":"true"%' AS has_defer_option, - options LIKE '%"flow_type":"batching"%' AS has_flow_type_option -FROM INFORMATION_SCHEMA.FLOWS -WHERE flow_name = 'pending_with_defer'; - -INSERT INTO pending_source VALUES (10, 0), (11, 1), (12, 2); - --- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | -ADMIN FLUSH_FLOW('pending_with_defer'); - -SELECT val FROM pending_sink ORDER BY val; - -CREATE OR REPLACE FLOW pending_with_defer -SINK TO pending_sink -WITH (defer_on_missing_source = true) -AS SELECT val FROM pending_replacement_source WHERE val > 100; - -SELECT - flow_definition, - source_table_ids, - source_table_names, - flownode_ids, - options LIKE '%"defer_on_missing_source":"true"%' AS has_defer_option, - options LIKE '%"flow_type":"batching"%' AS has_flow_type_option -FROM INFORMATION_SCHEMA.FLOWS -WHERE flow_name = 'pending_with_defer'; - -CREATE TABLE pending_replacement_source ( - val INT, - ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, - TIME INDEX(ts) -); - --- SQLNESS SLEEP 12s - -INSERT INTO pending_replacement_source VALUES (99, 3), (101, 4), (102, 5); - --- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED | -ADMIN FLUSH_FLOW('pending_with_defer'); - -SELECT val FROM pending_sink ORDER BY val; - DROP FLOW pending_with_defer; -DROP TABLE pending_sink; - -DROP TABLE pending_source; - -DROP TABLE pending_replacement_source; +SELECT flow_name FROM INFORMATION_SCHEMA.FLOWS WHERE flow_name = 'pending_with_defer'; From ec436a4eac92c904d45c33c227bfca8bd7048572 Mon Sep 17 00:00:00 2001 From: discord9 Date: Thu, 21 May 2026 17:07:59 +0800 Subject: [PATCH 3/4] test: cover pending flow metadata edge cases Signed-off-by: discord9 --- src/common/meta/src/ddl/create_flow.rs | 24 ++- src/common/meta/src/ddl/tests/create_flow.rs | 194 +++++++++++++++++++ src/operator/src/statement/ddl.rs | 5 +- 3 files changed, 219 insertions(+), 4 deletions(-) diff --git a/src/common/meta/src/ddl/create_flow.rs b/src/common/meta/src/ddl/create_flow.rs index a4fe5b62ec70..14a217a4d301 100644 --- a/src/common/meta/src/ddl/create_flow.rs +++ b/src/common/meta/src/ddl/create_flow.rs @@ -91,6 +91,8 @@ impl CreateFlowProcedure { let create_if_not_exists = self.data.task.create_if_not_exists; let or_replace = self.data.task.or_replace; + validate_flow_options(&self.data.task)?; + let flow_name_value = self .context .flow_metadata_manager @@ -431,6 +433,26 @@ pub fn defer_on_missing_source(flow_task: &CreateFlowTask) -> Result { .map(|value| value.unwrap_or(false)) } +pub fn validate_flow_options(flow_task: &CreateFlowTask) -> Result<()> { + for key in flow_task.flow_options.keys() { + match key.as_str() { + DEFER_ON_MISSING_SOURCE_KEY | FlowType::FLOW_TYPE_KEY => {} + unknown => { + return UnexpectedSnafu { + err_msg: format!( + "Unknown flow option '{unknown}', supported user options: {DEFER_ON_MISSING_SOURCE_KEY}" + ), + } + .fail(); + } + } + } + + defer_on_missing_source(flow_task)?; + get_flow_type_from_options(flow_task)?; + Ok(()) +} + fn user_runtime_flow_options(options: &HashMap) -> HashMap { let mut options = options.clone(); options.remove(DEFER_ON_MISSING_SOURCE_KEY); @@ -572,7 +594,7 @@ impl From<&CreateFlowData> for (FlowInfoValue, Vec<(FlowPartitionId, FlowRouteVa .collect::>(); let flow_type = value.flow_type.unwrap_or_default().to_string(); - options.insert("flow_type".to_string(), flow_type); + options.insert(FlowType::FLOW_TYPE_KEY.to_string(), flow_type); let mut create_time = chrono::Utc::now(); if let Some(prev_flow_value) = value.prev_flow_info_value.as_ref() diff --git a/src/common/meta/src/ddl/tests/create_flow.rs b/src/common/meta/src/ddl/tests/create_flow.rs index a04d5b692a23..19c56d52750b 100644 --- a/src/common/meta/src/ddl/tests/create_flow.rs +++ b/src/common/meta/src/ddl/tests/create_flow.rs @@ -134,6 +134,101 @@ async fn test_create_pending_flow_source_table_not_found_with_defer() { ); } +#[tokio::test] +async fn test_create_pending_flow_source_table_not_found_with_defer_false() { + let source_table_names = vec![TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "my_table", + )]; + let sink_table_name = + TableName::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "my_sink_table"); + let mut task = test_create_flow_task("my_flow", source_table_names, sink_table_name, false); + task.flow_options + .insert(DEFER_ON_MISSING_SOURCE_KEY.to_string(), "false".to_string()); + let node_manager = Arc::new(MockFlownodeManager::new(NaiveFlownodeHandler)); + let ddl_context = new_ddl_context(node_manager); + let query_ctx = test_query_context(); + let mut procedure = CreateFlowProcedure::new(task, query_ctx, ddl_context); + let err = procedure.on_prepare().await.unwrap_err(); + assert_matches!(err, error::Error::Unsupported { .. }); + assert!( + err.to_string() + .contains("requires WITH ('defer_on_missing_source'='true')") + ); +} + +#[tokio::test] +async fn test_create_pending_flow_records_partial_source_resolution() { + let existing_source = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "partial_existing_source_table", + ); + let missing_source = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "partial_missing_source_table", + ); + let sink_table_name = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "partial_pending_sink_table", + ); + let node_manager = Arc::new(MockFlownodeManager::new(NaiveFlownodeHandler)); + let ddl_context = new_ddl_context(node_manager); + + let existing_table_id = 3026; + let create_table_task = + test_create_table_task("partial_existing_source_table", existing_table_id); + ddl_context + .table_metadata_manager + .create_table_metadata( + create_table_task.table_info.clone(), + TableRouteValue::physical(vec![]), + HashMap::new(), + ) + .await + .unwrap(); + + let mut task = test_create_flow_task( + "partial_pending_flow", + vec![existing_source.clone(), missing_source.clone()], + sink_table_name, + false, + ); + enable_defer_on_missing_source(&mut task); + let query_ctx = test_query_context(); + let mut procedure = CreateFlowProcedure::new(task, query_ctx, ddl_context.clone()); + let status = procedure.on_prepare().await.unwrap(); + assert_matches!(status, Status::Executing { persist: true, .. }); + assert_eq!(procedure.data.source_table_ids, vec![existing_table_id]); + assert_eq!( + procedure.data.unresolved_source_table_names, + vec![missing_source.clone()] + ); + + let output = execute_procedure_until_done(&mut procedure).await.unwrap(); + let flow_id = *output.downcast_ref::().unwrap(); + let flow_info = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(flow_id) + .await + .unwrap() + .unwrap(); + + assert!(flow_info.is_pending()); + assert_eq!(flow_info.source_table_ids(), &[existing_table_id]); + let expected_all_sources = vec![existing_source, missing_source.clone()]; + assert_eq!( + flow_info.all_source_table_names(), + expected_all_sources.as_slice() + ); + assert_eq!(flow_info.unresolved_source_table_names(), &[missing_source]); + assert!(flow_info.flownode_ids().is_empty()); +} + #[test] fn test_defer_on_missing_source_defaults_false() { let task = test_create_flow_task( @@ -180,6 +275,29 @@ fn test_defer_on_missing_source_invalid_value() { ); } +#[tokio::test] +async fn test_create_flow_rejects_unknown_option_in_meta_task() { + let mut task = test_create_flow_task( + "my_flow", + vec![], + TableName::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "my_sink_table"), + false, + ); + task.flow_options + .insert("unknown_option".to_string(), "value".to_string()); + let node_manager = Arc::new(MockFlownodeManager::new(NaiveFlownodeHandler)); + let ddl_context = new_ddl_context(node_manager); + let query_ctx = test_query_context(); + let mut procedure = CreateFlowProcedure::new(task, query_ctx, ddl_context); + + let err = procedure.on_prepare().await.unwrap_err(); + assert_matches!(err, error::Error::Unexpected { .. }); + assert!( + err.to_string() + .contains("Unknown flow option 'unknown_option'") + ); +} + #[test] fn test_create_request_strips_defer_on_missing_source_runtime_option() { let mut task = test_create_flow_task( @@ -440,6 +558,82 @@ async fn test_replace_active_flow_with_pending_flow_is_unsupported() { ); } +#[tokio::test] +async fn test_replace_pending_flow_with_pending_flow_updates_metadata() { + let first_missing_source = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "replace_pending_first_missing_source", + ); + let second_missing_source = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "replace_pending_second_missing_source", + ); + let sink_table_name = TableName::new( + DEFAULT_CATALOG_NAME, + DEFAULT_SCHEMA_NAME, + "replace_pending_to_pending_sink_table", + ); + let node_manager = Arc::new(MockFlownodeManager::new(NaiveFlownodeHandler)); + let ddl_context = new_ddl_context(node_manager); + + let original_flow_id = create_test_pending_flow( + &ddl_context, + "replace_pending_to_pending_flow", + vec![first_missing_source.clone()], + sink_table_name.clone(), + ) + .await; + + let original_flow = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(original_flow_id) + .await + .unwrap() + .unwrap(); + assert!(original_flow.is_pending()); + assert_eq!( + original_flow.unresolved_source_table_names(), + &[first_missing_source] + ); + assert!(original_flow.flownode_ids().is_empty()); + + let mut replace_task = test_create_flow_task( + "replace_pending_to_pending_flow", + vec![second_missing_source.clone()], + sink_table_name, + false, + ); + enable_defer_on_missing_source(&mut replace_task); + replace_task.or_replace = true; + let query_ctx = test_query_context(); + let mut procedure = CreateFlowProcedure::new(replace_task, query_ctx, ddl_context.clone()); + let output = execute_procedure_until_done(&mut procedure).await.unwrap(); + let replaced_flow_id = *output.downcast_ref::().unwrap(); + assert_eq!(replaced_flow_id, original_flow_id); + + let replaced_flow = ddl_context + .flow_metadata_manager + .flow_info_manager() + .get(replaced_flow_id) + .await + .unwrap() + .unwrap(); + assert!(replaced_flow.is_pending()); + assert_eq!(replaced_flow.source_table_ids(), Vec::::new()); + assert_eq!( + replaced_flow.unresolved_source_table_names(), + &[second_missing_source.clone()] + ); + assert_eq!( + replaced_flow.all_source_table_names(), + &[second_missing_source] + ); + assert!(replaced_flow.flownode_ids().is_empty()); +} + #[tokio::test] async fn test_create_flow_same_source_and_sink_table() { let table_id = 1024; diff --git a/src/operator/src/statement/ddl.rs b/src/operator/src/statement/ddl.rs index f8d40271fffc..0dab4148bb4b 100644 --- a/src/operator/src/statement/ddl.rs +++ b/src/operator/src/statement/ddl.rs @@ -34,7 +34,7 @@ use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, is_reado use common_catalog::{format_full_flow_name, format_full_table_name}; use common_error::ext::BoxedError; use common_meta::cache_invalidator::Context; -use common_meta::ddl::create_flow::FlowType; +use common_meta::ddl::create_flow::{DEFER_ON_MISSING_SOURCE_KEY, FlowType}; use common_meta::instruction::CacheIdent; use common_meta::key::schema_name::{SchemaName, SchemaNameKey}; use common_meta::procedure_executor::ExecutorContext; @@ -113,7 +113,6 @@ struct DdlSubmitOptions { timeout: Duration, } -const DEFER_ON_MISSING_SOURCE_KEY: &str = "defer_on_missing_source"; const ALLOWED_FLOW_OPTIONS: [&str; 1] = [DEFER_ON_MISSING_SOURCE_KEY]; fn build_procedure_id_output(procedure_id: Vec) -> Result { @@ -224,7 +223,7 @@ fn determine_flow_type_for_source_state( } ); info!( - "Flow `{}` defaults to batching because defer_on_missing_source=true and some source tables are not available yet", + "Flow `{}` is created as a pending batching flow because source tables are missing and defer_on_missing_source=true", flow_name ); return Ok(Some(FlowType::Batching)); From 2f73dce03f87814c03227c931c4edd9e3ad176dc Mon Sep 17 00:00:00 2001 From: discord9 Date: Thu, 21 May 2026 17:10:21 +0800 Subject: [PATCH 4/4] test: fix pending flow metadata test lint Signed-off-by: discord9 --- src/common/meta/src/ddl/tests/create_flow.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/common/meta/src/ddl/tests/create_flow.rs b/src/common/meta/src/ddl/tests/create_flow.rs index 19c56d52750b..a1a6c040f18f 100644 --- a/src/common/meta/src/ddl/tests/create_flow.rs +++ b/src/common/meta/src/ddl/tests/create_flow.rs @@ -625,7 +625,7 @@ async fn test_replace_pending_flow_with_pending_flow_updates_metadata() { assert_eq!(replaced_flow.source_table_ids(), Vec::::new()); assert_eq!( replaced_flow.unresolved_source_table_names(), - &[second_missing_source.clone()] + std::slice::from_ref(&second_missing_source) ); assert_eq!( replaced_flow.all_source_table_names(),