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

refactor: remove ouput_exprs from LogicalKafkaScan #16385

Merged
merged 4 commits into from
Apr 19, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
2 changes: 1 addition & 1 deletion src/frontend/src/handler/create_source.rs
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ use crate::handler::create_table::{
use crate::handler::util::SourceSchemaCompatExt;
use crate::handler::HandlerArgs;
use crate::optimizer::plan_node::generic::SourceNodeKind;
use crate::optimizer::plan_node::{LogicalSource, ToStream, ToStreamContext};
use crate::optimizer::plan_node::{LogicalSource, ToStreamContext};
use crate::session::SessionImpl;
use crate::utils::resolve_privatelink_in_with_option;
use crate::{bind_data_type, build_graph, OptimizerContext, WithOptions};
Expand Down
5 changes: 2 additions & 3 deletions src/frontend/src/handler/create_table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -708,15 +708,14 @@ fn gen_table_plan_inner(
});

let source_catalog = source.as_ref().map(|source| Rc::new((source).into()));
let source_node: PlanRef = LogicalSource::new(
let source_node: PlanRef = LogicalSource::create(
source_catalog.clone(),
columns.clone(),
row_id_index,
SourceNodeKind::CreateTable,
context.clone(),
None,
)?
.into();
)?;

let required_cols = FixedBitSet::with_capacity(columns.len());
let mut plan_root = PlanRoot::new(
Expand Down
4 changes: 2 additions & 2 deletions src/frontend/src/optimizer/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ use crate::expr::TimestamptzExprFinder;
use crate::optimizer::plan_node::generic::{SourceNodeKind, Union};
use crate::optimizer::plan_node::{
BatchExchange, PlanNodeType, PlanTreeNode, RewriteExprsRecursive, StreamExchange, StreamUnion,
ToStream, VisitExprsRecursive,
VisitExprsRecursive,
};
use crate::optimizer::plan_visitor::TemporalJoinValidator;
use crate::optimizer::property::Distribution;
Expand Down Expand Up @@ -637,7 +637,7 @@ impl PlanRoot {
}
};

let dummy_source_node = LogicalSource::new(
let dummy_source_node = LogicalSource::create(
None,
columns.clone(),
row_id_index,
Expand Down
49 changes: 6 additions & 43 deletions src/frontend/src/optimizer/plan_node/logical_kafka_scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,12 @@ use risingwave_connector::source::DataType;
use super::generic::GenericPlanRef;
use super::utils::{childless_record, Distill};
use super::{
generic, BatchProject, ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject,
PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream,
generic, ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject, PlanBase,
PlanRef, PredicatePushdown, ToBatch, ToStream,
};
use crate::catalog::source_catalog::SourceCatalog;
use crate::error::Result;
use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor};
use crate::expr::{Expr, ExprImpl, ExprType};
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::plan_node::utils::column_names_pretty;
use crate::optimizer::plan_node::{
Expand All @@ -44,10 +44,6 @@ pub struct LogicalKafkaScan {
pub base: PlanBase<Logical>,
pub core: generic::Source,

/// Expressions to output. This field presents and will be turned to a `Project` when
/// converting to a physical plan, only if there are generated columns.
output_exprs: Option<Vec<ExprImpl>>,

/// Kafka timestamp range.
kafka_timestamp_range: (Bound<i64>, Bound<i64>),
}
Expand All @@ -63,7 +59,6 @@ impl LogicalKafkaScan {
LogicalKafkaScan {
base,
core,
output_exprs: logical_source.output_exprs.clone(),
kafka_timestamp_range,
}
}
Expand All @@ -76,7 +71,6 @@ impl LogicalKafkaScan {
Self {
base: self.base.clone(),
core: self.core.clone(),
output_exprs: self.output_exprs.clone(),
kafka_timestamp_range: range,
}
}
Expand Down Expand Up @@ -107,34 +101,9 @@ impl ColPrunable for LogicalKafkaScan {
}
}

impl ExprRewritable for LogicalKafkaScan {
fn has_rewritable_expr(&self) -> bool {
self.output_exprs.is_some()
}

fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef {
let mut output_exprs = self.output_exprs.clone();
impl ExprRewritable for LogicalKafkaScan {}

for expr in output_exprs.iter_mut().flatten() {
*expr = r.rewrite_expr(expr.clone());
}

Self {
output_exprs,
..self.clone()
}
.into()
}
}

impl ExprVisitable for LogicalKafkaScan {
fn visit_exprs(&self, v: &mut dyn ExprVisitor) {
self.output_exprs
.iter()
.flatten()
.for_each(|e| v.visit_expr(e));
}
}
impl ExprVisitable for LogicalKafkaScan {}

/// A util function to extract kafka offset timestamp range.
///
Expand Down Expand Up @@ -320,14 +289,8 @@ impl PredicatePushdown for LogicalKafkaScan {

impl ToBatch for LogicalKafkaScan {
fn to_batch(&self) -> Result<PlanRef> {
let mut plan: PlanRef =
let plan: PlanRef =
BatchKafkaScan::new(self.core.clone(), self.kafka_timestamp_range).into();

if let Some(exprs) = &self.output_exprs {
let logical_project = generic::Project::new(exprs.to_vec(), plan);
plan = BatchProject::new(logical_project).into();
}

Ok(plan)
}
}
Expand Down
75 changes: 21 additions & 54 deletions src/frontend/src/optimizer/plan_node/logical_source.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,13 +28,13 @@ use super::generic::{GenericPlanRef, SourceNodeKind};
use super::stream_watermark_filter::StreamWatermarkFilter;
use super::utils::{childless_record, Distill};
use super::{
generic, BatchProject, BatchSource, ColPrunable, ExprRewritable, Logical, LogicalFilter,
LogicalProject, PlanBase, PlanRef, PredicatePushdown, StreamProject, StreamRowIdGen,
StreamSource, StreamSourceScan, ToBatch, ToStream,
generic, BatchSource, ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject,
PlanBase, PlanRef, PredicatePushdown, StreamRowIdGen, StreamSource, StreamSourceScan, ToBatch,
ToStream,
};
use crate::catalog::source_catalog::SourceCatalog;
use crate::error::Result;
use crate::expr::{ExprImpl, ExprRewriter, ExprVisitor, InputRef};
use crate::expr::{ExprImpl, ExprRewriter, InputRef};
use crate::optimizer::optimizer_context::OptimizerContextRef;
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::plan_node::stream_fs_fetch::StreamFsFetch;
Expand All @@ -53,23 +53,20 @@ pub struct LogicalSource {
pub base: PlanBase<Logical>,
pub core: generic::Source,

/// Expressions to output. This field presents and will be turned to a `Project` when
/// converting to a physical plan, only if there are generated columns.
pub(crate) output_exprs: Option<Vec<ExprImpl>>,
/// When there are generated columns, the `StreamRowIdGen`'s `row_id_index` is different from
/// the one in `core`. So we store the one in `output_exprs` here.
pub(crate) output_row_id_index: Option<usize>,
}

impl LogicalSource {
pub fn new(
pub fn create(
source_catalog: Option<Rc<SourceCatalog>>,
column_catalog: Vec<ColumnCatalog>,
row_id_index: Option<usize>,
kind: SourceNodeKind,
ctx: OptimizerContextRef,
as_of: Option<AsOf>,
) -> Result<Self> {
) -> Result<PlanRef> {
let core = generic::Source {
catalog: source_catalog,
column_catalog,
Expand All @@ -83,32 +80,37 @@ impl LogicalSource {
bail!("Time travel is not supported for the source")
}

let base = PlanBase::new_logical_with_core(&core);

let output_exprs = Self::derive_output_exprs_from_generated_columns(&core.column_catalog)?;
let (core, output_row_id_index) = core.exclude_generated_columns();

Ok(LogicalSource {
let base = PlanBase::new_logical_with_core(&core);

let source = LogicalSource {
base,
core,
output_exprs,
output_row_id_index,
})
};

if let Some(exprs) = output_exprs {
Ok(LogicalProject::create(source.into(), exprs.to_vec()))
} else {
Ok(source.into())
}
}

pub fn with_catalog(
source_catalog: Rc<SourceCatalog>,
kind: SourceNodeKind,
ctx: OptimizerContextRef,
as_of: Option<AsOf>,
) -> Result<Self> {
) -> Result<PlanRef> {
let column_catalogs = source_catalog.columns.clone();
let row_id_index = source_catalog.row_id_index;
if !source_catalog.append_only {
assert!(row_id_index.is_none());
}

Self::new(
Self::create(
Some(source_catalog),
column_catalogs,
row_id_index,
Expand Down Expand Up @@ -270,34 +272,9 @@ impl ColPrunable for LogicalSource {
}
}

impl ExprRewritable for LogicalSource {
fn has_rewritable_expr(&self) -> bool {
self.output_exprs.is_some()
}

fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef {
let mut output_exprs = self.output_exprs.clone();
impl ExprRewritable for LogicalSource {}

for expr in output_exprs.iter_mut().flatten() {
*expr = r.rewrite_expr(expr.clone());
}

Self {
output_exprs,
..self.clone()
}
.into()
}
}

impl ExprVisitable for LogicalSource {
fn visit_exprs(&self, v: &mut dyn ExprVisitor) {
self.output_exprs
.iter()
.flatten()
.for_each(|e| v.visit_expr(e));
}
}
impl ExprVisitable for LogicalSource {}

impl PredicatePushdown for LogicalSource {
fn predicate_pushdown(
Expand All @@ -315,12 +292,7 @@ impl ToBatch for LogicalSource {
!self.core.is_kafka_connector(),
"LogicalSource with a kafka property should be converted to LogicalKafkaScan"
);
let mut plan: PlanRef = BatchSource::new(self.core.clone()).into();

if let Some(exprs) = &self.output_exprs {
let logical_project = generic::Project::new(exprs.to_vec(), plan);
plan = BatchProject::new(logical_project).into();
}
let plan: PlanRef = BatchSource::new(self.core.clone()).into();

Ok(plan)
}
Expand Down Expand Up @@ -357,11 +329,6 @@ impl ToStream for LogicalSource {
}
}

if let Some(exprs) = &self.output_exprs {
let logical_project = generic::Project::new(exprs.to_vec(), plan);
plan = StreamProject::new(logical_project).into();
}

if let Some(catalog) = self.source_catalog()
&& !catalog.watermark_descs.is_empty()
{
Expand Down
3 changes: 1 addition & 2 deletions src/frontend/src/planner/relation.rs
Original file line number Diff line number Diff line change
Expand Up @@ -125,8 +125,7 @@ impl Planner {
SourceNodeKind::CreateMViewOrBatch,
self.ctx(),
as_of,
)?
.into())
)?)
}
}

Expand Down
Loading