Skip to content

feat: support streaming iceberg source #20568

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

Merged
merged 36 commits into from
Apr 1, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
36 commits
Select commit Hold shift + click to select a range
d4c2f35
.
xxchan Mar 14, 2025
4fb2aeb
fix delete
xxchan Mar 17, 2025
1446b03
try
xxchan Mar 18, 2025
e411b81
bump
xxchan Mar 18, 2025
6c3e9b0
refine test
xxchan Mar 18, 2025
f213f41
/
xxchan Mar 18, 2025
92106aa
fix test
xxchan Mar 19, 2025
e93fc52
test recovery
xxchan Mar 19, 2025
860b36c
Merge remote-tracking branch 'origin/main' into xxchan/iceberg-stream…
xxchan Mar 20, 2025
f95101a
try refactored incremental scan
xxchan Mar 20, 2025
d06c2db
Merge remote-tracking branch 'origin/main' into xxchan/iceberg-stream…
xxchan Mar 20, 2025
a659ebe
fix compie
xxchan Mar 20, 2025
f873dec
Merge remote-tracking branch 'origin/main' into xxchan/iceberg-stream…
xxchan Mar 21, 2025
f54ef0c
fix FETCH & add large chunk test
xxchan Mar 21, 2025
1d5517d
bump
xxchan Mar 21, 2025
f60644b
smaller data
xxchan Mar 21, 2025
47b2abf
avoid writing too many files.
xxchan Mar 21, 2025
b3202da
fix
xxchan Mar 21, 2025
78244fe
Merge branch 'main' into xxchan/iceberg-stream-source
xxchan Mar 24, 2025
b32b02e
Merge remote-tracking branch 'origin/main' into xxchan/iceberg-stream…
xxchan Mar 25, 2025
465cb08
Merge remote-tracking branch 'origin/main' into xxchan/iceberg-stream…
xxchan Mar 26, 2025
7dae0aa
- dead code
xxchan Mar 26, 2025
4c84b48
Merge remote-tracking branch 'origin/main' into xxchan/iceberg-stream…
xxchan Mar 26, 2025
e405eba
fix
xxchan Mar 26, 2025
e630f30
fix example.toml
xxchan Mar 26, 2025
8fcaf2e
compatibility layer
xxchan Mar 26, 2025
ddb98d5
fix
xxchan Mar 26, 2025
f89aa1a
fix
xxchan Mar 26, 2025
98805d5
fix
xxchan Mar 26, 2025
ee2e049
mod
xxchan Mar 26, 2025
79a16f9
fix
xxchan Mar 26, 2025
88ab290
Merge remote-tracking branch 'origin/main' into xxchan/iceberg-stream…
xxchan Mar 27, 2025
3ddbaa5
Merge remote-tracking branch 'origin/main' into xxchan/iceberg-stream…
xxchan Mar 27, 2025
252079c
Merge branch 'main' into xxchan/iceberg-stream-source
xxchan Mar 31, 2025
d4c4ae7
revert exactly once for table engine
wcy-fdu Apr 1, 2025
6142571
Merge remote-tracking branch 'origin/wcy/revert.pr' into xxchan/icebe…
xxchan Apr 1, 2025
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 Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

215 changes: 215 additions & 0 deletions e2e_test/iceberg/test_case/iceberg_source_streaming.slt
Original file line number Diff line number Diff line change
@@ -0,0 +1,215 @@
statement ok
SET RW_IMPLICIT_FLUSH = true;

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

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

statement ok
CREATE SINK sink1 AS select * from mv1 WITH (
connector = 'iceberg',
type = 'upsert',
database.name = 'demo_db',
table.name = 'test_streaming',
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',
create_table_if_not_exists = 'true',
commit_checkpoint_interval = 1,
primary_key = 'i1,i2',
partition_by = 'i1'
);

statement ok
INSERT INTO s1 (i1, i2, i3) values(1,'1','1'),(2,'2','2'),(3,'3','3'),(4,'4','4'),(5,'5','5');

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',
catalog.type = 'storage',
warehouse.path = 's3a://icebergdata/demo',
database.name = 'demo_db',
table.name = 'test_streaming',
);

statement ok
create materialized view mv2 as select * from iceberg_t1_source;

# select only some columns
statement ok
create materialized view mv3 as select i2, i1 * 3 as abc from iceberg_t1_source;

sleep 5s

query I retry 3 backoff 5s
select * from mv2 order by i1;
----
1 1 1
2 2 2
3 3 3
4 4 4
5 5 5


query I retry 3 backoff 5s
select * from mv3 order by i2;
----
1 3
2 6
3 9
4 12
5 15


# delete cannot be reflected
statement ok
DELETE FROM s1 WHERE i1 < 3;

sleep 5s

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

query I retry 3 backoff 5s
select count(*) from mv2;
----
5

query I retry 3 backoff 5s
select count(*) from mv3;
----
5


# insert more data
statement ok
INSERT INTO s1 (i1, i2, i3) values(6,'6','6'),(7,'7','7'),(8,'8','8'),(9,'9','9'),(10,'10','10');

sleep 5s

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

query I retry 3 backoff 5s
select count(*) from mv2;
----
10

query I retry 3 backoff 5s
select count(*) from mv3;
----
10


# insert more data -- with upsert
statement ok
INSERT INTO s1 (i1, i2, i3) values(9,'9','99'),(10,'10','1010'), (11,'11','1111');

sleep 5s

query I
select * from iceberg_t1_source order by i1;
----
3 3 3
4 4 4
5 5 5
6 6 6
7 7 7
8 8 8
9 9 99
10 10 1010
11 11 1111

query I retry 3 backoff 5s
select * from mv2 order by i1, i3;
----
1 1 1
2 2 2
3 3 3
4 4 4
5 5 5
6 6 6
7 7 7
8 8 8
9 9 9
9 9 99
10 10 10
10 10 1010
11 11 1111

query I retry 3 backoff 5s
select count(*) from mv3;
----
13

# test recovery
statement ok
RECOVER;

# insert more data
statement ok
INSERT INTO s1 (i1, i2, i3) values(12,'12','1212'),(13,'13','1313'),(14,'14','1414'),(15,'15','1515'),(16,'16','1616');

sleep 5s

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

query I retry 3 backoff 5s
select count(*) from mv2;
----
18

query I retry 3 backoff 5s
select count(*) from mv3;
----
18

# insert large batch of data
statement ok
INSERT INTO s1 (i1, i2, i3) select 1, i::varchar, i::varchar from generate_series(10001, 20000) as i;

sleep 5s

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

query I retry 3 backoff 5s
select count(*) from mv2;
----
10018

query I retry 3 backoff 5s
select count(*) from mv3;
----
10018

# TODO: also test compaction

statement ok
DROP SINK sink1;

statement ok
DROP SOURCE iceberg_t1_source CASCADE;

statement ok
DROP TABLE s1 cascade;
11 changes: 11 additions & 0 deletions e2e_test/iceberg/test_case/iceberg_source_streaming.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.test_streaming PURGE',
]

slt = 'test_case/iceberg_source_streaming.slt'

drop_sqls = [
'DROP TABLE IF EXISTS demo_db.test_streaming PURGE',
'DROP SCHEMA IF EXISTS demo_db',
]
8 changes: 4 additions & 4 deletions src/batch/executors/src/executor/iceberg_scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ pub struct IcebergScanExecutor {
#[allow(dead_code)]
snapshot_id: Option<i64>,
file_scan_tasks: Option<IcebergFileScanTask>,
batch_size: usize,
chunk_size: usize,
schema: Schema,
identity: String,
metrics: Option<BatchMetrics>,
Expand All @@ -66,7 +66,7 @@ impl IcebergScanExecutor {
iceberg_config: IcebergProperties,
snapshot_id: Option<i64>,
file_scan_tasks: IcebergFileScanTask,
batch_size: usize,
chunk_size: usize,
schema: Schema,
identity: String,
metrics: Option<BatchMetrics>,
Expand All @@ -76,7 +76,7 @@ impl IcebergScanExecutor {
Self {
iceberg_config,
snapshot_id,
batch_size,
chunk_size,
schema,
file_scan_tasks: Some(file_scan_tasks),
identity,
Expand Down Expand Up @@ -113,7 +113,7 @@ impl IcebergScanExecutor {
table.clone(),
data_file_scan_task,
IcebergScanOpts {
batch_size: self.batch_size,
chunk_size: self.chunk_size,
need_seq_num: self.need_seq_num,
need_file_path_and_pos: self.need_file_path_and_pos,
},
Expand Down
4 changes: 4 additions & 0 deletions src/common/src/catalog/column.rs
Original file line number Diff line number Diff line change
Expand Up @@ -418,6 +418,10 @@ impl ColumnCatalog {
)),
]
}

pub fn is_row_id_column(&self) -> bool {
self.column_desc.column_id == ROW_ID_COLUMN_ID
}
}

impl From<PbColumnCatalog> for ColumnCatalog {
Expand Down
24 changes: 24 additions & 0 deletions src/common/src/config.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1232,6 +1232,18 @@ pub struct StreamingDeveloperConfig {

#[serde(default)]
pub compute_client_config: RpcClientConfig,

/// `IcebergListExecutor`: The interval in seconds for Iceberg source to list new files.
#[serde(default = "default::developer::iceberg_list_interval_sec")]
pub iceberg_list_interval_sec: u64,

/// `IcebergFetchExecutor`: The number of files the executor will fetch concurrently in a batch.
#[serde(default = "default::developer::iceberg_fetch_batch_size")]
pub iceberg_fetch_batch_size: u64,

/// `IcebergSink`: The size of the cache for positional delete in the sink.
#[serde(default = "default::developer::iceberg_sink_positional_delete_cache_size")]
pub iceberg_sink_positional_delete_cache_size: usize,
}

/// The subsections `[batch.developer]`.
Expand Down Expand Up @@ -2244,6 +2256,18 @@ pub mod default {
pub fn rpc_client_connect_timeout_secs() -> u64 {
5
}

pub fn iceberg_list_interval_sec() -> u64 {
1
}

pub fn iceberg_fetch_batch_size() -> u64 {
1024
}

pub fn iceberg_sink_positional_delete_cache_size() -> usize {
1024
}
}

pub use crate::system_param::default as system;
Expand Down
3 changes: 3 additions & 0 deletions src/config/example.toml
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,9 @@ stream_switch_jdbc_pg_to_native = false
stream_max_barrier_batch_size = 1024
stream_hash_join_entry_state_max_rows = 30000
stream_enable_explain_analyze_stats = true
stream_iceberg_list_interval_sec = 1
stream_iceberg_fetch_batch_size = 1024
stream_iceberg_sink_positional_delete_cache_size = 1024

[streaming.developer.stream_compute_client_config]
connect_timeout_secs = 5
Expand Down
5 changes: 4 additions & 1 deletion src/connector/src/sink/iceberg/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -840,7 +840,10 @@ impl IcebergSinkWriter {
MonitoredPositionDeleteWriterBuilder::new(
SortPositionDeleteWriterBuilder::new(
parquet_writer_builder.clone(),
1024,
writer_param
.streaming_config
.developer
.iceberg_sink_positional_delete_cache_size,
None,
None,
),
Expand Down
3 changes: 3 additions & 0 deletions src/connector/src/sink/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@ use prometheus::Registry;
use risingwave_common::array::ArrayError;
use risingwave_common::bitmap::Bitmap;
use risingwave_common::catalog::{ColumnDesc, Field, Schema};
use risingwave_common::config::StreamingConfig;
use risingwave_common::hash::ActorId;
use risingwave_common::metrics::{
LabelGuardedHistogram, LabelGuardedHistogramVec, LabelGuardedIntCounter,
Expand Down Expand Up @@ -491,6 +492,7 @@ pub struct SinkWriterParam {
pub sink_id: SinkId,
pub sink_name: String,
pub connector: String,
pub streaming_config: StreamingConfig,
}

#[derive(Clone)]
Expand Down Expand Up @@ -586,6 +588,7 @@ impl SinkWriterParam {
sink_id: SinkId::new(1),
sink_name: "test_sink".to_owned(),
connector: "test_connector".to_owned(),
streaming_config: StreamingConfig::default(),
}
}
}
Expand Down
Loading
Loading