Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: support background DDL for MV on shared source #18919

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions src/frontend/src/optimizer/plan_node/stream_materialize.rs
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ use crate::catalog::table_catalog::{TableCatalog, TableType, TableVersion};
use crate::error::Result;
use crate::optimizer::plan_node::derive::derive_pk;
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::plan_node::utils::plan_has_backfill_leaf_nodes;
use crate::optimizer::plan_node::utils::plan_can_use_backgronud_ddl;
use crate::optimizer::plan_node::{PlanBase, PlanNodeMeta};
use crate::optimizer::property::{Cardinality, Distribution, Order, RequiredDist};
use crate::stream_fragmenter::BuildFragmentGraphState;
Expand Down Expand Up @@ -88,7 +88,7 @@ impl StreamMaterialize {

let create_type = if matches!(table_type, TableType::MaterializedView)
&& input.ctx().session_ctx().config().background_ddl()
&& plan_has_backfill_leaf_nodes(&input)
&& plan_can_use_backgronud_ddl(&input)
{
CreateType::Background
} else {
Expand Down
4 changes: 2 additions & 2 deletions src/frontend/src/optimizer/plan_node/stream_sink.rs
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ use super::{generic, ExprRewritable, PlanBase, PlanRef, StreamNode, StreamProjec
use crate::error::{ErrorCode, Result};
use crate::expr::{ExprImpl, FunctionCall, InputRef};
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::plan_node::utils::plan_has_backfill_leaf_nodes;
use crate::optimizer::plan_node::utils::plan_can_use_backgronud_ddl;
use crate::optimizer::plan_node::PlanTreeNodeUnary;
use crate::optimizer::property::{Distribution, Order, RequiredDist};
use crate::stream_fragmenter::BuildFragmentGraphState;
Expand Down Expand Up @@ -380,7 +380,7 @@ impl StreamSink {
let input = required_dist.enforce_if_not_satisfies(input, &Order::any())?;
let distribution_key = input.distribution().dist_column_indices().to_vec();
let create_type = if input.ctx().session_ctx().config().background_ddl()
&& plan_has_backfill_leaf_nodes(&input)
&& plan_can_use_backgronud_ddl(&input)
{
CreateType::Background
} else {
Expand Down
10 changes: 7 additions & 3 deletions src/frontend/src/optimizer/plan_node/utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -377,17 +377,21 @@ pub fn infer_kv_log_store_table_catalog_inner(

/// Check that all leaf nodes must be stream table scan,
/// since that plan node maps to `backfill` executor, which supports recovery.
pub(crate) fn plan_has_backfill_leaf_nodes(plan: &PlanRef) -> bool {
/// Some other leaf nodes like `StreamValues` do not support recovery, and they
/// cannot use background ddl.
Comment on lines +380 to +381
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we have an issue for this? Is it possible to address this by simply recording the state into a state table?

pub(crate) fn plan_can_use_backgronud_ddl(plan: &PlanRef) -> bool {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

typo background

if plan.inputs().is_empty() {
if let Some(scan) = plan.as_stream_table_scan() {
if plan.as_stream_source_scan().is_some() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For stream_source_scan it is only used by source backfill right? If so this LGTM.

true
} else if let Some(scan) = plan.as_stream_table_scan() {
scan.stream_scan_type() == StreamScanType::Backfill
|| scan.stream_scan_type() == StreamScanType::ArrangementBackfill
} else {
false
}
} else {
assert!(!plan.inputs().is_empty());
plan.inputs().iter().all(plan_has_backfill_leaf_nodes)
plan.inputs().iter().all(plan_can_use_backgronud_ddl)
}
}

Expand Down
1 change: 1 addition & 0 deletions src/meta/src/rpc/ddl_controller_v2.rs
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@ impl DdlController {
tracing::debug!(
id = job_id,
definition = streaming_job.definition(),
create_type = streaming_job.create_type().as_str_name(),
"starting streaming job",
);
let _permit = self
Expand Down
Loading