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(optimizer): add logical and batch iceberg scan operator #16402

Merged
merged 2 commits into from
Apr 22, 2024
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
5 changes: 4 additions & 1 deletion src/frontend/src/optimizer/logical_optimization.rs
Original file line number Diff line number Diff line change
Expand Up @@ -427,7 +427,10 @@ static LOGICAL_FILTER_EXPRESSION_SIMPLIFY: LazyLock<OptimizationStage> = LazyLoc
static REWRITE_SOURCE_FOR_BATCH: LazyLock<OptimizationStage> = LazyLock::new(|| {
OptimizationStage::new(
"Rewrite Source For Batch",
vec![SourceToKafkaScanRule::create()],
vec![
SourceToKafkaScanRule::create(),
SourceToIcebergScanRule::create(),
],
ApplyOrder::TopDown,
)
});
Expand Down
2 changes: 2 additions & 0 deletions src/frontend/src/optimizer/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -975,6 +975,7 @@ fn require_additional_exchange_on_root_in_distributed_mode(plan: PlanRef) -> boo
fn is_source(plan: &PlanRef) -> bool {
plan.node_type() == PlanNodeType::BatchSource
|| plan.node_type() == PlanNodeType::BatchKafkaScan
|| plan.node_type() == PlanNodeType::BatchIcebergScan
}

fn is_insert(plan: &PlanRef) -> bool {
Expand Down Expand Up @@ -1007,6 +1008,7 @@ fn require_additional_exchange_on_root_in_local_mode(plan: PlanRef) -> bool {
fn is_source(plan: &PlanRef) -> bool {
plan.node_type() == PlanNodeType::BatchSource
|| plan.node_type() == PlanNodeType::BatchKafkaScan
|| plan.node_type() == PlanNodeType::BatchIcebergScan
}

fn is_insert(plan: &PlanRef) -> bool {
Expand Down
118 changes: 118 additions & 0 deletions src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,118 @@
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::rc::Rc;

use pretty_xmlish::{Pretty, XmlNode};
use risingwave_pb::batch_plan::plan_node::NodeBody;
use risingwave_pb::batch_plan::SourceNode;
use risingwave_sqlparser::ast::AsOf;

use super::batch::prelude::*;
use super::utils::{childless_record, column_names_pretty, Distill};
use super::{
generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch,
};
use crate::catalog::source_catalog::SourceCatalog;
use crate::error::Result;
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::property::{Distribution, Order};

#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct BatchIcebergScan {
pub base: PlanBase<Batch>,
pub core: generic::Source,
}

impl BatchIcebergScan {
pub fn new(core: generic::Source) -> Self {
let base = PlanBase::new_batch_with_core(
&core,
// Use `Single` by default, will be updated later with `clone_with_dist`.
Distribution::Single,
Order::any(),
);

Self { base, core }
}

pub fn column_names(&self) -> Vec<&str> {
self.schema().names_str()
}

pub fn source_catalog(&self) -> Option<Rc<SourceCatalog>> {
self.core.catalog.clone()
}

pub fn clone_with_dist(&self) -> Self {
let base = self
.base
.clone_with_new_distribution(Distribution::SomeShard);
Self {
base,
core: self.core.clone(),
}
}

pub fn as_of(&self) -> Option<AsOf> {
self.core.as_of.clone()
}
}

impl_plan_tree_node_for_leaf! { BatchIcebergScan }

impl Distill for BatchIcebergScan {
fn distill<'a>(&self) -> XmlNode<'a> {
let src = Pretty::from(self.source_catalog().unwrap().name.clone());
let fields = vec![
("source", src),
("columns", column_names_pretty(self.schema())),
];
childless_record("BatchIcebergScan", fields)
}
}

impl ToLocalBatch for BatchIcebergScan {
fn to_local(&self) -> Result<PlanRef> {
Ok(self.clone_with_dist().into())
}
}

impl ToDistributedBatch for BatchIcebergScan {
fn to_distributed(&self) -> Result<PlanRef> {
Ok(self.clone_with_dist().into())
}
}

impl ToBatchPb for BatchIcebergScan {
fn to_batch_prost_body(&self) -> NodeBody {
let source_catalog = self.source_catalog().unwrap();
NodeBody::Source(SourceNode {
source_id: source_catalog.id,
info: Some(source_catalog.info.clone()),
columns: self
.core
.column_catalog
.iter()
.map(|c| c.to_protobuf())
.collect(),
with_properties: source_catalog.with_properties.clone().into_iter().collect(),
split: vec![],
})
}
}

impl ExprRewritable for BatchIcebergScan {}

impl ExprVisitable for BatchIcebergScan {}
116 changes: 116 additions & 0 deletions src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::rc::Rc;

use pretty_xmlish::{Pretty, XmlNode};

use super::generic::GenericPlanRef;
use super::utils::{childless_record, Distill};
use super::{
generic, ColPrunable, ExprRewritable, Logical, LogicalProject, PlanBase, PlanRef,
PredicatePushdown, ToBatch, ToStream,
};
use crate::catalog::source_catalog::SourceCatalog;
use crate::error::Result;
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::plan_node::utils::column_names_pretty;
use crate::optimizer::plan_node::{
BatchIcebergScan, ColumnPruningContext, LogicalFilter, LogicalSource, PredicatePushdownContext,
RewriteStreamContext, ToStreamContext,
};
use crate::utils::{ColIndexMapping, Condition};

/// `LogicalIcebergScan` is only used by batch queries. At the beginning of the batch query optimization, `LogicalSource` with a iceberg property would be converted into a `LogicalIcebergScan`.
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct LogicalIcebergScan {
pub base: PlanBase<Logical>,
pub core: generic::Source,
}

impl LogicalIcebergScan {
pub fn new(logical_source: &LogicalSource) -> Self {
assert!(logical_source.core.is_iceberg_connector());

let core = logical_source.core.clone();
let base = PlanBase::new_logical_with_core(&core);

assert!(logical_source.output_exprs.is_none());

LogicalIcebergScan { base, core }
}

pub fn source_catalog(&self) -> Option<Rc<SourceCatalog>> {
self.core.catalog.clone()
}
}

impl_plan_tree_node_for_leaf! {LogicalIcebergScan}
impl Distill for LogicalIcebergScan {
fn distill<'a>(&self) -> XmlNode<'a> {
let fields = if let Some(catalog) = self.source_catalog() {
let src = Pretty::from(catalog.name.clone());
vec![
("source", src),
("columns", column_names_pretty(self.schema())),
]
} else {
vec![]
};
childless_record("LogicalIcebergScan", fields)
}
}

impl ColPrunable for LogicalIcebergScan {
fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef {
// TODO: support column pruning for iceberg scan
let mapping = ColIndexMapping::with_remaining_columns(required_cols, self.schema().len());
LogicalProject::with_mapping(self.clone().into(), mapping).into()
}
}

impl ExprRewritable for LogicalIcebergScan {}

impl ExprVisitable for LogicalIcebergScan {}

impl PredicatePushdown for LogicalIcebergScan {
fn predicate_pushdown(
&self,
predicate: Condition,
_ctx: &mut PredicatePushdownContext,
) -> PlanRef {
// No pushdown.
LogicalFilter::create(self.clone().into(), predicate)
}
}

impl ToBatch for LogicalIcebergScan {
fn to_batch(&self) -> Result<PlanRef> {
let plan: PlanRef = BatchIcebergScan::new(self.core.clone()).into();
Ok(plan)
}
}

impl ToStream for LogicalIcebergScan {
fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result<PlanRef> {
unreachable!()
}

fn logical_rewrite_for_stream(
&self,
_ctx: &mut RewriteStreamContext,
) -> Result<(PlanRef, ColIndexMapping)> {
unreachable!()
}
}
4 changes: 4 additions & 0 deletions src/frontend/src/optimizer/plan_node/logical_source.rs
Original file line number Diff line number Diff line change
Expand Up @@ -315,6 +315,10 @@ impl ToBatch for LogicalSource {
!self.core.is_kafka_connector(),
"LogicalSource with a kafka property should be converted to LogicalKafkaScan"
);
assert!(
!self.core.is_iceberg_connector(),
"LogicalSource with a iceberg property should be converted to LogicalIcebergScan"
);
let mut plan: PlanRef = BatchSource::new(self.core.clone()).into();

if let Some(exprs) = &self.output_exprs {
Expand Down
8 changes: 8 additions & 0 deletions src/frontend/src/optimizer/plan_node/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -904,8 +904,10 @@ mod stream_topn;
mod stream_values;
mod stream_watermark_filter;

mod batch_iceberg_scan;
mod batch_kafka_scan;
mod derive;
mod logical_iceberg_scan;
mod stream_cdc_table_scan;
mod stream_share;
mod stream_temporal_join;
Expand All @@ -920,6 +922,7 @@ pub use batch_group_topn::BatchGroupTopN;
pub use batch_hash_agg::BatchHashAgg;
pub use batch_hash_join::BatchHashJoin;
pub use batch_hop_window::BatchHopWindow;
pub use batch_iceberg_scan::BatchIcebergScan;
pub use batch_insert::BatchInsert;
pub use batch_kafka_scan::BatchKafkaScan;
pub use batch_limit::BatchLimit;
Expand Down Expand Up @@ -949,6 +952,7 @@ pub use logical_except::LogicalExcept;
pub use logical_expand::LogicalExpand;
pub use logical_filter::LogicalFilter;
pub use logical_hop_window::LogicalHopWindow;
pub use logical_iceberg_scan::LogicalIcebergScan;
pub use logical_insert::LogicalInsert;
pub use logical_intersect::LogicalIntersect;
pub use logical_join::LogicalJoin;
Expand Down Expand Up @@ -1058,6 +1062,7 @@ macro_rules! for_all_plan_nodes {
, { Logical, Except }
, { Logical, MaxOneRow }
, { Logical, KafkaScan }
, { Logical, IcebergScan }
, { Batch, SimpleAgg }
, { Batch, HashAgg }
, { Batch, SortAgg }
Expand Down Expand Up @@ -1086,6 +1091,7 @@ macro_rules! for_all_plan_nodes {
, { Batch, OverWindow }
, { Batch, MaxOneRow }
, { Batch, KafkaScan }
, { Batch, IcebergScan }
, { Stream, Project }
, { Stream, Filter }
, { Stream, TableScan }
Expand Down Expand Up @@ -1158,6 +1164,7 @@ macro_rules! for_logical_plan_nodes {
, { Logical, Except }
, { Logical, MaxOneRow }
, { Logical, KafkaScan }
, { Logical, IcebergScan }
}
};
}
Expand Down Expand Up @@ -1195,6 +1202,7 @@ macro_rules! for_batch_plan_nodes {
, { Batch, OverWindow }
, { Batch, MaxOneRow }
, { Batch, KafkaScan }
, { Batch, IcebergScan }
}
};
}
Expand Down
4 changes: 4 additions & 0 deletions src/frontend/src/optimizer/rule/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -157,10 +157,13 @@ pub use apply_hop_window_transpose_rule::*;
mod agg_call_merge_rule;
pub use agg_call_merge_rule::*;
mod pull_up_correlated_predicate_agg_rule;
mod source_to_iceberg_scan_rule;
mod source_to_kafka_scan_rule;
mod values_extract_project_rule;

pub use batch::batch_push_limit_to_scan_rule::*;
pub use pull_up_correlated_predicate_agg_rule::*;
pub use source_to_iceberg_scan_rule::*;
pub use source_to_kafka_scan_rule::*;
pub use values_extract_project_rule::*;

Expand Down Expand Up @@ -236,6 +239,7 @@ macro_rules! for_all_rules {
, { BatchPushLimitToScanRule }
, { PullUpCorrelatedPredicateAggRule }
, { SourceToKafkaScanRule }
, { SourceToIcebergScanRule }
}
};
}
Expand Down
35 changes: 35 additions & 0 deletions src/frontend/src/optimizer/rule/source_to_iceberg_scan_rule.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use super::{BoxedRule, Rule};
use crate::optimizer::plan_node::{LogicalIcebergScan, LogicalSource};
use crate::optimizer::PlanRef;

pub struct SourceToIcebergScanRule {}
impl Rule for SourceToIcebergScanRule {
fn apply(&self, plan: PlanRef) -> Option<PlanRef> {
let source: &LogicalSource = plan.as_logical_source()?;
if source.core.is_iceberg_connector() {
Some(LogicalIcebergScan::new(source).into())
} else {
None
}
}
}

impl SourceToIcebergScanRule {
pub fn create() -> BoxedRule {
Box::new(SourceToIcebergScanRule {})
}
}
Loading
Loading