Skip to content

Commit

Permalink
refactor(frontend): derive create_type when generating the table ca…
Browse files Browse the repository at this point in the history
…talog (#16827)
  • Loading branch information
kwannoel authored May 20, 2024
1 parent 8b5a7bc commit 5ba1874
Show file tree
Hide file tree
Showing 5 changed files with 45 additions and 31 deletions.
31 changes: 4 additions & 27 deletions src/frontend/src/handler/create_mv.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,8 @@ use either::Either;
use itertools::Itertools;
use pgwire::pg_response::{PgResponse, StatementType};
use risingwave_common::acl::AclMode;
use risingwave_pb::catalog::{CreateType, PbTable};
use risingwave_pb::catalog::PbTable;
use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism;
use risingwave_pb::stream_plan::StreamScanType;
use risingwave_sqlparser::ast::{EmitMode, Ident, ObjectName, Query};

use super::privilege::resolve_relation_privileges;
Expand Down Expand Up @@ -167,7 +166,7 @@ pub async fn handle_create_mv(
return Ok(resp);
}

let (mut table, graph, can_run_in_background) = {
let (table, graph) = {
let context = OptimizerContext::from_handler_args(handler_args);
if !context.with_options().is_empty() {
// get other useful fields by `remove`, the logic here is to reject unknown options.
Expand All @@ -186,21 +185,7 @@ It only indicates the physical clustering of the data, which may improve the per

let (plan, table) =
gen_create_mv_plan(&session, context.into(), query, name, columns, emit_mode)?;
// All leaf nodes must be stream table scan, no other scan operators support recovery.
fn plan_has_backfill_leaf_nodes(plan: &PlanRef) -> bool {
if plan.inputs().is_empty() {
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)
}
}
let can_run_in_background = plan_has_backfill_leaf_nodes(&plan);

let context = plan.plan_base().ctx().clone();
let mut graph = build_graph(plan)?;
graph.parallelism =
Expand All @@ -214,7 +199,7 @@ It only indicates the physical clustering of the data, which may improve the per
let ctx = graph.ctx.as_mut().unwrap();
ctx.timezone = context.get_session_timezone();

(table, graph, can_run_in_background)
(table, graph)
};

// Ensure writes to `StreamJobTracker` are atomic.
Expand All @@ -229,14 +214,6 @@ It only indicates the physical clustering of the data, which may improve the per
table.name.clone(),
));

let run_in_background = session.config().background_ddl();
let create_type = if run_in_background && can_run_in_background {
CreateType::Background
} else {
CreateType::Foreground
};
table.create_type = create_type.into();

let session = session.clone();
let catalog_writer = session.catalog_writer()?;
catalog_writer
Expand Down
14 changes: 13 additions & 1 deletion src/frontend/src/optimizer/plan_node/stream_materialize.rs
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +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::{PlanBase, PlanNodeMeta};
use crate::optimizer::property::{Cardinality, Distribution, Order, RequiredDist};
use crate::stream_fragmenter::BuildFragmentGraphState;
Expand Down Expand Up @@ -84,6 +85,14 @@ impl StreamMaterialize {
let input = reorganize_elements_id(input);
let columns = derive_columns(input.schema(), out_names, &user_cols)?;

let create_type = if matches!(table_type, TableType::MaterializedView)
&& input.ctx().session_ctx().config().background_ddl()
&& plan_has_backfill_leaf_nodes(&input)
{
CreateType::Background
} else {
CreateType::Foreground
};
let table = Self::derive_table_catalog(
input.clone(),
name,
Expand All @@ -98,6 +107,7 @@ impl StreamMaterialize {
None,
cardinality,
retention_seconds,
create_type,
)?;

Ok(Self::new(input, table))
Expand Down Expand Up @@ -139,6 +149,7 @@ impl StreamMaterialize {
version,
Cardinality::unknown(), // unknown cardinality for tables
retention_seconds,
CreateType::Foreground,
)?;

Ok(Self::new(input, table))
Expand Down Expand Up @@ -210,6 +221,7 @@ impl StreamMaterialize {
version: Option<TableVersion>,
cardinality: Cardinality,
retention_seconds: Option<NonZeroU32>,
create_type: CreateType,
) -> Result<TableCatalog> {
let input = rewritten_input;

Expand Down Expand Up @@ -259,7 +271,7 @@ impl StreamMaterialize {
created_at_epoch: None,
initialized_at_epoch: None,
cleaned_by_watermark: false,
create_type: CreateType::Foreground, // Will be updated in the handler itself.
create_type,
stream_job_status: StreamJobStatus::Creating,
description: None,
incoming_sinks: vec![],
Expand Down
5 changes: 4 additions & 1 deletion src/frontend/src/optimizer/plan_node/stream_sink.rs
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,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::PlanTreeNodeUnary;
use crate::optimizer::property::{Distribution, Order, RequiredDist};
use crate::stream_fragmenter::BuildFragmentGraphState;
Expand Down Expand Up @@ -373,7 +374,9 @@ 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() {
let create_type = if input.ctx().session_ctx().config().background_ddl()
&& plan_has_backfill_leaf_nodes(&input)
{
CreateType::Background
} else {
CreateType::Foreground
Expand Down
17 changes: 17 additions & 0 deletions src/frontend/src/optimizer/plan_node/utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
use crate::catalog::table_catalog::TableType;
use crate::catalog::{ColumnId, TableCatalog, TableId};
use crate::optimizer::property::{Cardinality, Order, RequiredDist};
use crate::optimizer::StreamScanType;
use crate::utils::{Condition, IndexSet};

#[derive(Default)]
Expand Down Expand Up @@ -370,3 +371,19 @@ pub fn infer_kv_log_store_table_catalog_inner(

table_catalog_builder.build(dist_key, read_prefix_len_hint)
}

/// 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 {
if plan.inputs().is_empty() {
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)
}
}
9 changes: 7 additions & 2 deletions src/meta/src/rpc/ddl_controller.rs
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,8 @@ use crate::manager::{
CatalogManagerRef, ConnectionId, DatabaseId, FragmentManagerRef, FunctionId, IdCategory,
IdCategoryType, IndexId, LocalNotification, MetaSrvEnv, MetadataManager, MetadataManagerV1,
NotificationVersion, RelationIdEnum, SchemaId, SinkId, SourceId, StreamingClusterInfo,
StreamingJob, SubscriptionId, TableId, UserId, ViewId, IGNORED_NOTIFICATION_VERSION,
StreamingJob, StreamingJobDiscriminants, SubscriptionId, TableId, UserId, ViewId,
IGNORED_NOTIFICATION_VERSION,
};
use crate::model::{FragmentId, StreamContext, TableFragments, TableParallelism};
use crate::rpc::cloud_provider::AwsEc2Client;
Expand Down Expand Up @@ -909,7 +910,7 @@ impl DdlController {
)
.await
}
(CreateType::Background, _) => {
(CreateType::Background, &StreamingJob::MaterializedView(_)) => {
let ctrl = self.clone();
let mgr = mgr.clone();
let stream_job_id = stream_job.id();
Expand All @@ -935,6 +936,10 @@ impl DdlController {
tokio::spawn(fut);
Ok(IGNORED_NOTIFICATION_VERSION)
}
(CreateType::Background, _) => {
let d: StreamingJobDiscriminants = stream_job.into();
bail!("background_ddl not supported for: {:?}", d)
}
}
}

Expand Down

0 comments on commit 5ba1874

Please sign in to comment.