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(iceberg): Separate iceberg source pb from source pb #18209

Merged
merged 6 commits into from
Aug 27, 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
13 changes: 6 additions & 7 deletions proto/batch_plan.proto
Original file line number Diff line number Diff line change
Expand Up @@ -66,12 +66,11 @@ message SourceNode {
map<string, secret.SecretRef> secret_refs = 6;
}

message IcebergSourceNode {
uint32 source_id = 1;
repeated plan_common.ColumnCatalog columns = 2;
map<string, string> with_properties = 3;
repeated bytes split = 4;
map<string, secret.SecretRef> secret_refs = 5;
message IcebergScanNode {
repeated plan_common.ColumnCatalog columns = 1;
map<string, string> with_properties = 2;
repeated bytes split = 3;
map<string, secret.SecretRef> secret_refs = 4;
}

message FileScanNode {
Expand Down Expand Up @@ -373,7 +372,7 @@ message PlanNode {
MaxOneRowNode max_one_row = 36;
LogRowSeqScanNode log_row_seq_scan = 37;
FileScanNode file_scan = 38;
IcebergSourceNode iceberg_source = 39;
IcebergScanNode iceberg_scan = 39;
// The following nodes are used for testing.
bool block_executor = 100;
bool busy_loop_executor = 101;
Expand Down
2 changes: 1 addition & 1 deletion src/batch/src/executor/iceberg_scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -131,7 +131,7 @@ impl BoxedExecutorBuilder for IcebergScanExecutorBuilder {
);
let source_node = try_match_expand!(
source.plan_node().get_node_body().unwrap(),
NodeBody::IcebergSource
NodeBody::IcebergScan
)?;

// prepare connector source
Expand Down
2 changes: 1 addition & 1 deletion src/batch/src/executor/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -243,7 +243,7 @@ impl<'a, C: BatchTaskContext> ExecutorBuilder<'a, C> {
NodeBody::SortOverWindow => SortOverWindowExecutor,
NodeBody::MaxOneRow => MaxOneRowExecutor,
NodeBody::FileScan => FileScanExecutorBuilder,
NodeBody::IcebergSource => IcebergScanExecutorBuilder,
NodeBody::IcebergScan => IcebergScanExecutorBuilder,
// Follow NodeBody only used for test
NodeBody::BlockExecutor => BlockExecutorBuilder,
NodeBody::BusyLoopExecutor => BusyLoopExecutorBuilder,
Expand Down
5 changes: 2 additions & 3 deletions src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ use std::rc::Rc;

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

use super::batch::prelude::*;
Expand Down Expand Up @@ -99,8 +99,7 @@ impl ToBatchPb for BatchIcebergScan {
fn to_batch_prost_body(&self) -> NodeBody {
let source_catalog = self.source_catalog().unwrap();
let (with_properties, secret_refs) = source_catalog.with_properties.clone().into_parts();
NodeBody::IcebergSource(IcebergSourceNode {
source_id: source_catalog.id,
NodeBody::IcebergScan(IcebergScanNode {
columns: self
.core
.column_catalog
Expand Down
56 changes: 32 additions & 24 deletions src/frontend/src/scheduler/distributed/stage.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1052,36 +1052,44 @@ impl StageRunner {
node_body: Some(NodeBody::LogRowSeqScan(scan_node)),
}
}
PlanNodeType::BatchSource
| PlanNodeType::BatchKafkaScan
| PlanNodeType::BatchIcebergScan => {
PlanNodeType::BatchSource | PlanNodeType::BatchKafkaScan => {
let node_body = execution_plan_node.node.clone();
let NodeBody::Source(mut source_node) = node_body else {
unreachable!();
};

let partition = partition
.expect("no partition info for seq scan")
.into_source()
.expect("PartitionInfo should be SourcePartitionInfo");
if let NodeBody::Source(mut source_node) = node_body {
source_node.split = partition
.into_iter()
.map(|split| split.encode_to_bytes().into())
.collect_vec();
PbPlanNode {
children: vec![],
identity,
node_body: Some(NodeBody::Source(source_node)),
}
} else if let NodeBody::IcebergSource(mut source_node) = node_body {
source_node.split = partition
.into_iter()
.map(|split| split.encode_to_bytes().into())
.collect_vec();
PbPlanNode {
children: vec![],
identity,
node_body: Some(NodeBody::IcebergSource(source_node)),
}
} else {
source_node.split = partition
.into_iter()
.map(|split| split.encode_to_bytes().into())
.collect_vec();
PbPlanNode {
children: vec![],
identity,
node_body: Some(NodeBody::Source(source_node)),
}
}
PlanNodeType::BatchIcebergScan => {
let node_body = execution_plan_node.node.clone();
let NodeBody::IcebergScan(mut iceberg_source_node) = node_body else {
Copy link
Contributor

Choose a reason for hiding this comment

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

iceberg_source_node => iceberg_scan_node

unreachable!();
};

let partition = partition
.expect("no partition info for seq scan")
.into_source()
.expect("PartitionInfo should be SourcePartitionInfo");
iceberg_source_node.split = partition
.into_iter()
.map(|split| split.encode_to_bytes().into())
.collect_vec();
PbPlanNode {
children: vec![],
identity,
node_body: Some(NodeBody::IcebergScan(iceberg_source_node)),
}
}
_ => {
Expand Down
41 changes: 23 additions & 18 deletions src/frontend/src/scheduler/local.rs
Original file line number Diff line number Diff line change
Expand Up @@ -554,12 +554,10 @@ impl LocalQueryExecution {
node_body: Some(node_body),
})
}
PlanNodeType::BatchSource
| PlanNodeType::BatchKafkaScan
| PlanNodeType::BatchIcebergScan => {
let node_body = execution_plan_node.node.clone();
match node_body {
NodeBody::Source(mut source_node) => {
PlanNodeType::BatchSource | PlanNodeType::BatchKafkaScan => {
let mut node_body = execution_plan_node.node.clone();
match &mut node_body {
NodeBody::Source(ref mut source_node) => {
if let Some(partition) = partition {
let partition = partition
.into_source()
Expand All @@ -569,13 +567,20 @@ impl LocalQueryExecution {
.map(|split| split.encode_to_bytes().into())
.collect_vec();
}
Ok(PbPlanNode {
children: vec![],
identity,
node_body: Some(NodeBody::Source(source_node)),
})
}
NodeBody::IcebergSource(mut iceberg_source_node) => {
_ => unreachable!(),
}

Ok(PbPlanNode {
children: vec![],
identity,
node_body: Some(node_body),
})
}
PlanNodeType::BatchIcebergScan => {
let mut node_body = execution_plan_node.node.clone();
match &mut node_body {
NodeBody::IcebergScan(ref mut iceberg_source_node) => {
Copy link
Contributor

Choose a reason for hiding this comment

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

iceberg_source_node => iceberg_scan_node

if let Some(partition) = partition {
let partition = partition
.into_source()
Expand All @@ -585,15 +590,15 @@ impl LocalQueryExecution {
.map(|split| split.encode_to_bytes().into())
.collect_vec();
}

Ok(PbPlanNode {
children: vec![],
identity,
node_body: Some(NodeBody::IcebergSource(iceberg_source_node)),
})
}
_ => unreachable!(),
}

Ok(PbPlanNode {
children: vec![],
identity,
node_body: Some(node_body),
})
}
PlanNodeType::BatchLookupJoin => {
let mut node_body = execution_plan_node.node.clone();
Expand Down
2 changes: 1 addition & 1 deletion src/prost/build.rs
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ fn main() -> Result<(), Box<dyn std::error::Error>> {
".stream_plan.SourceBackfillNode",
".stream_plan.StreamSource",
".batch_plan.SourceNode",
".batch_plan.IcebergSourceNode",
".batch_plan.IcebergScanNode",
];

// Build protobuf structs.
Expand Down
Loading