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(frontend): support iceberg predicate pushdown #19228

Open
wants to merge 39 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
39 commits
Select commit Hold shift + click to select a range
a0656ff
match rw predicates
kwannoel Oct 28, 2024
a105859
mark place to add filter
kwannoel Oct 29, 2024
7016979
pass in schema fields as a parameter
kwannoel Nov 1, 2024
b0c00c0
convert input ref to reference, datum to iceberg datum
kwannoel Nov 1, 2024
f07e235
convert rw expressions into iceberg predicates
kwannoel Nov 2, 2024
aa7e899
add support for more literals
kwannoel Nov 2, 2024
9b9bac0
add predicate proto
kwannoel Nov 4, 2024
31dff2f
interim commit: use iceberg proto
kwannoel Nov 4, 2024
11d5930
change predicate_pushdown return
kwannoel Nov 4, 2024
f648af3
derive eq, hash for iceberg predicate
kwannoel Nov 4, 2024
eed2ed2
interim commit: add iceberg_predicate to batch
kwannoel Nov 4, 2024
ed6f73d
add fetch_parameters
kwannoel Nov 5, 2024
5fb182b
Revert "derive eq, hash for iceberg predicate"
kwannoel Nov 6, 2024
6dea2bc
Revert "change predicate_pushdown return"
kwannoel Nov 6, 2024
33b6103
Revert "interim commit: use iceberg proto"
kwannoel Nov 6, 2024
b53d205
Revert "add predicate proto"
kwannoel Nov 6, 2024
287b032
Revert "interim commit: add iceberg_predicate to batch"
kwannoel Nov 6, 2024
060b6f1
use iceberg predicate in logical_iceberg_scan fields
kwannoel Nov 6, 2024
1e23059
add to batch
kwannoel Nov 6, 2024
34392c8
build with predicate
kwannoel Nov 6, 2024
1dc7fd5
clean
kwannoel Nov 6, 2024
28389ee
implement distill
kwannoel Nov 7, 2024
57f0c33
fix warn
kwannoel Nov 7, 2024
6f41c2e
add tests
kwannoel Nov 7, 2024
0c9da00
no verbose for wget
kwannoel Nov 7, 2024
693b608
more tests
kwannoel Nov 7, 2024
e76d65d
check results
kwannoel Nov 7, 2024
5672443
fix bugs
kwannoel Nov 7, 2024
d742365
explain source plan, maybe schema malformed
kwannoel Nov 7, 2024
519c115
fix tests
kwannoel Nov 8, 2024
f4e9633
fmt
kwannoel Nov 8, 2024
32823b5
increase timeout
kwannoel Nov 8, 2024
665b46f
no need double assert
kwannoel Nov 8, 2024
0d2f661
docs
kwannoel Nov 8, 2024
a7c98a2
fmt
kwannoel Nov 8, 2024
f394302
increase timeout
kwannoel Nov 8, 2024
115e2ef
use rule based predicate push down
kwannoel Nov 8, 2024
e60b4e4
prune BatchFilter if predicate always true
kwannoel Nov 8, 2024
326cd17
test mix filter and predicate
kwannoel Nov 8, 2024
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
1 change: 1 addition & 0 deletions ci/scripts/e2e-iceberg-sink-v2-test.sh
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ poetry run python main.py -t ./test_case/iceberg_select_empty_table.toml
poetry run python main.py -t ./test_case/iceberg_source_equality_delete.toml
poetry run python main.py -t ./test_case/iceberg_source_position_delete.toml
poetry run python main.py -t ./test_case/iceberg_source_all_delete.toml
poetry run python main.py -t ./test_case/iceberg_predicate_pushdown.toml


echo "--- Kill cluster"
Expand Down
3 changes: 1 addition & 2 deletions ci/workflows/main-cron.yml
Original file line number Diff line number Diff line change
Expand Up @@ -414,14 +414,13 @@ steps:
depends_on:
- "build"
- "build-other"

plugins:
- docker-compose#v5.1.0:
run: rw-build-env
config: ci/docker-compose.yml
mount-buildkite-agent: true
- ./ci/plugins/upload-failure-logs
timeout_in_minutes: 7
timeout_in_minutes: 9
retry: *auto-retry

- label: "end-to-end iceberg sink v2 test (release)"
Expand Down
2 changes: 1 addition & 1 deletion ci/workflows/pull-request.yml
Original file line number Diff line number Diff line change
Expand Up @@ -270,7 +270,7 @@ steps:
config: ci/docker-compose.yml
mount-buildkite-agent: true
- ./ci/plugins/upload-failure-logs
timeout_in_minutes: 15
timeout_in_minutes: 17
retry: *auto-retry

- label: "end-to-end iceberg cdc test"
Expand Down
2 changes: 1 addition & 1 deletion e2e_test/iceberg/start_spark_connect_server.sh
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ PACKAGES="$PACKAGES,org.apache.spark:spark-connect_2.12:$SPARK_VERSION"
SPARK_FILE="spark-${SPARK_VERSION}-bin-hadoop3.tgz"

if [ ! -d "spark-${SPARK_VERSION}-bin-hadoop3" ];then
wget https://dlcdn.apache.org/spark/spark-${SPARK_VERSION}/$SPARK_FILE
wget --no-verbose https://dlcdn.apache.org/spark/spark-${SPARK_VERSION}/$SPARK_FILE
tar -xzf $SPARK_FILE --no-same-owner
fi

Expand Down
138 changes: 138 additions & 0 deletions e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
statement ok
set sink_decouple = false;

statement ok
set streaming_parallelism=4;

statement ok
drop table if exists s1 cascade;

statement ok
CREATE TABLE s1 (i1 int, i2 varchar, i3 varchar);

statement ok
insert into s1 select x, 'some str', 'another str' from generate_series(1, 500) t(x);

statement ok
insert into s1 select x, null as y, null as z from generate_series(501, 1000) t(x);

statement ok
flush;

statement ok
CREATE MATERIALIZED VIEW mv1 AS SELECT * FROM s1;

statement ok
CREATE SINK sink1 AS select * from mv1 WITH (
connector = 'iceberg',
type = 'append-only',
force_append_only = 'true',
database.name = 'demo_db',
table.name = 't1',
catalog.name = 'demo',
catalog.type = 'storage',
warehouse.path = 's3a://icebergdata/demo',
s3.endpoint = 'http://127.0.0.1:9301',
s3.region = 'us-east-1',
s3.access.key = 'hummockadmin',
s3.secret.key = 'hummockadmin',
commit_checkpoint_interval = 1,
create_table_if_not_exists = 'true'
);

statement ok
drop source if exists iceberg_t1_source;

statement ok
CREATE SOURCE iceberg_t1_source
WITH (
connector = 'iceberg',
s3.endpoint = 'http://127.0.0.1:9301',
s3.region = 'us-east-1',
s3.access.key = 'hummockadmin',
s3.secret.key = 'hummockadmin',
s3.path.style.access = 'true',
catalog.type = 'storage',
warehouse.path = 's3a://icebergdata/demo',
database.name = 'demo_db',
table.name = 't1',
);

statement ok
flush;

query I
select * from iceberg_t1_source order by i1 limit 1;
----
1 some str another str

query I
select count(*) from iceberg_t1_source;
----
1000

query I
select * from iceberg_t1_source where i1 > 990 order by i1;
----
991 NULL NULL
992 NULL NULL
993 NULL NULL
994 NULL NULL
995 NULL NULL
996 NULL NULL
997 NULL NULL
998 NULL NULL
999 NULL NULL
1000 NULL NULL

query I
explain select * from iceberg_t1_source where i1 > 500 and i1 < 600 and i1 >= 550 and i1 <= 590 and i1 != 570 and i1 = 580;
----
BatchExchange { order: [], dist: Single }
└─BatchIcebergScan { source: iceberg_t1_source, columns: [i1, i2, i3], predicate: (((((i1 = 580) AND (i1 > 500)) AND (i1 < 600)) AND (i1 >= 550)) AND (i1 <= 590)) AND (i1 != 570) }

query I
select i1 from iceberg_t1_source where i1 > 500 and i1 < 600 and i1 >= 550 and i1 <= 590 and i1 != 570 and i1 = 580;
----
580

query I
explain select * from iceberg_t1_source where i1 in (1, 2, 3, 4, 5);
----
BatchExchange { order: [], dist: Single }
└─BatchIcebergScan { source: iceberg_t1_source, columns: [i1, i2, i3], predicate: i1 IN (5, 4, 1, 3, 2) }

query I
select i1 from iceberg_t1_source where i1 in (1, 2, 3, 4, 5) order by i1;
----
1
2
3
4
5

query I
select count(*), i2, i3 from iceberg_t1_source where i2 = 'some str' and i3 = 'another str' group by i2, i3;
----
500 some str another str

query I
explain select i1 from iceberg_t1_source where i1 > 500 and i2 = i3;
----
BatchExchange { order: [], dist: Single }
└─BatchProject { exprs: [i1] }
└─BatchFilter { predicate: (i2 = i3) }
└─BatchIcebergScan { source: iceberg_t1_source, columns: [i1, i2, i3], predicate: i1 > 500 }

query I
select i1 from iceberg_t1_source where i1 > 500 and i2 = i3;
----

statement ok
DROP SINK sink1;

statement ok
DROP SOURCE iceberg_t1_source;

statement ok
DROP TABLE s1 cascade;
11 changes: 11 additions & 0 deletions e2e_test/iceberg/test_case/iceberg_predicate_pushdown.toml
Original file line number Diff line number Diff line change
@@ -0,0 +1,11 @@
init_sqls = [
'CREATE SCHEMA IF NOT EXISTS demo_db',
'DROP TABLE IF EXISTS demo_db.t1',
]

slt = 'test_case/iceberg_predicate_pushdown.slt'

drop_sqls = [
'DROP TABLE IF EXISTS demo_db.t1',
'DROP SCHEMA IF EXISTS demo_db',
]
16 changes: 14 additions & 2 deletions src/connector/src/source/iceberg/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ use std::collections::HashMap;
use anyhow::anyhow;
use async_trait::async_trait;
use futures_async_stream::for_await;
use iceberg::expr::Predicate as IcebergPredicate;
use iceberg::scan::FileScanTask;
use iceberg::spec::TableMetadata;
use iceberg::table::Table;
Expand Down Expand Up @@ -189,6 +190,7 @@ impl IcebergSplitEnumerator {
schema: Schema,
time_traval_info: Option<IcebergTimeTravelInfo>,
batch_parallelism: usize,
predicate: IcebergPredicate,
) -> ConnectorResult<Vec<IcebergSplit>> {
if batch_parallelism == 0 {
bail!("Batch parallelism is 0. Cannot split the iceberg files.");
Expand Down Expand Up @@ -246,11 +248,15 @@ impl IcebergSplitEnumerator {

let require_names = Self::get_require_field_names(&table, snapshot_id, &schema).await?;

let table_schema = table.metadata().current_schema();
tracing::debug!("iceberg_table_schema: {:?}", table_schema);

let mut position_delete_files = vec![];
let mut data_files = vec![];
let mut equality_delete_files = vec![];
let scan = table
.scan()
.with_filter(predicate)
.snapshot_id(snapshot_id)
.select(require_names)
.build()
Expand Down Expand Up @@ -302,10 +308,16 @@ impl IcebergSplitEnumerator {
.files
.push(data_files[split_num * split_size + i].clone());
}
Ok(splits
let splits = splits
.into_iter()
.filter(|split| !split.files.is_empty())
.collect_vec())
.collect_vec();

if splits.is_empty() {
bail!("No splits found for the iceberg table");
}
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm thinking if it's reasonable to have 0 splits found, e.g. when querying empty iceberg table.


Ok(splits)
}

/// The required field names are the intersection of the output shema and the equality delete columns.
Expand Down
8 changes: 8 additions & 0 deletions src/frontend/src/optimizer/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -402,6 +402,14 @@ impl PlanRoot {
ApplyOrder::BottomUp,
));

// For iceberg scan, we do iceberg predicate pushdown
// BatchFilter -> BatchIcebergScan
let plan = plan.optimize_by_rules(&OptimizationStage::new(
"Iceberg Predicate Pushdown",
vec![BatchIcebergPredicatePushDownRule::create()],
ApplyOrder::BottomUp,
));

assert_eq!(plan.convention(), Convention::Batch);
Ok(plan)
}
Expand Down
6 changes: 6 additions & 0 deletions src/frontend/src/optimizer/plan_node/batch_filter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,12 @@ impl BatchFilter {
pub fn predicate(&self) -> &Condition {
&self.core.predicate
}

pub fn clone_with_predicate(&self, predicate: Condition) -> Self {
let mut core = self.core.clone();
core.predicate = predicate;
Self::new(core)
}
}
impl_distill_by_unit!(BatchFilter, core, "BatchFilter");

Expand Down
23 changes: 21 additions & 2 deletions src/frontend/src/optimizer/plan_node/batch_iceberg_scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@

use std::rc::Rc;

use educe::Educe;
use iceberg::expr::Predicate as IcebergPredicate;
use pretty_xmlish::{Pretty, XmlNode};
use risingwave_pb::batch_plan::plan_node::NodeBody;
use risingwave_pb::batch_plan::IcebergScanNode;
Expand All @@ -29,10 +31,13 @@ use crate::error::Result;
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::property::{Distribution, Order};

#[derive(Debug, Clone, PartialEq, Eq, Hash)]
#[derive(Educe, Debug, Clone, PartialEq)]
#[educe(Eq, Hash)]
pub struct BatchIcebergScan {
pub base: PlanBase<Batch>,
pub core: generic::Source,
#[educe(Hash(ignore))]
pub predicate: IcebergPredicate,
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hash and Eq are only required for streaming share plan. But down the road we may support batch share plan. In that case every single batch plan node using Educe needs to be audited.

}

impl BatchIcebergScan {
Expand All @@ -44,7 +49,11 @@ impl BatchIcebergScan {
Order::any(),
);

Self { base, core }
Self {
base,
core,
predicate: IcebergPredicate::AlwaysTrue,
}
}

pub fn column_names(&self) -> Vec<&str> {
Expand All @@ -62,6 +71,15 @@ impl BatchIcebergScan {
Self {
base,
core: self.core.clone(),
predicate: self.predicate.clone(),
}
}

pub fn clone_with_predicate(&self, predicate: IcebergPredicate) -> Self {
Self {
base: self.base.clone(),
core: self.core.clone(),
predicate,
}
}

Expand All @@ -78,6 +96,7 @@ impl Distill for BatchIcebergScan {
let fields = vec![
("source", src),
("columns", column_names_pretty(self.schema())),
("predicate", Pretty::from(self.predicate.to_string())),
];
childless_record("BatchIcebergScan", fields)
}
Expand Down
Loading
Loading