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(batch,optimizer): support column pruning for iceberg source #16429

Merged
merged 2 commits into from
Apr 22, 2024
Merged
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
19 changes: 16 additions & 3 deletions src/frontend/src/optimizer/plan_node/logical_iceberg_scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,15 @@ impl LogicalIcebergScan {
pub fn source_catalog(&self) -> Option<Rc<SourceCatalog>> {
self.core.catalog.clone()
}

pub fn clone_with_required_cols(&self, required_cols: &[usize]) -> Self {
assert!(!required_cols.is_empty());
let mut core = self.core.clone();
core.column_catalog = required_cols.iter().map(|idx| core.column_catalog[*idx].clone()).collect();
let base = PlanBase::new_logical_with_core(&core);

LogicalIcebergScan { base, core }
}
}

impl_plan_tree_node_for_leaf! {LogicalIcebergScan}
Expand All @@ -74,9 +83,13 @@ impl Distill for LogicalIcebergScan {

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()
if required_cols.is_empty() {
let mapping = ColIndexMapping::with_remaining_columns(required_cols, self.schema().len());
// If reuqiured_cols is empty, we use the first column of iceberg to avoid the empty schema.
LogicalProject::with_mapping(self.clone_with_required_cols(&[0]).into() , mapping).into()
Copy link
Contributor

Choose a reason for hiding this comment

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

So iceberg scan does not support scan no fields? But the RW scan can do it (only the row count is meaningful in this case)

dev=> explain select 1 from t;
                QUERY PLAN
-------------------------------------------
 BatchExchange { order: [], dist: Single }
 └─BatchProject { exprs: [1:Int32] }
   └─BatchScan { table: t, columns: [] }
(3 rows)

Copy link
Contributor

Choose a reason for hiding this comment

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

This is a very corner case, but I am wondering if Iceberg supports creating a table without any columns. It should be possible in PG

postgres=# create table t();
CREATE TABLE
postgres=# \d+ t;
                                          Table "public.t"
 Column | Type | Collation | Nullable | Default | Storage | Compression | Stats target | Description
--------+------+-----------+----------+---------+---------+-------------+--------------+-------------
Access method: heap

Copy link
Contributor Author

@chenzl25 chenzl25 Apr 22, 2024

Choose a reason for hiding this comment

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

So iceberg scan does not support scan no fields? But the RW scan can do it (only the row count is meaningful in this case)

Actually, the underlying sdk icelake doesn't handle the empty schema well and it will treat empty schema as all columns are required, so we need to make a trick here.

This is a very corner case, but I am wondering if Iceberg supports creating a table without any columns. It should be possible in PG

Iceberg doesn't support creating a table without any columns.

} else {
self.clone_with_required_cols(required_cols).into()
}
}
}

Expand Down
Loading