Skip to content

Commit c947797

Browse files
committed
add support for creating new iceberg tables with partition keys
1 parent 0f5bae0 commit c947797

12 files changed

+79
-7
lines changed

e2e_test/iceberg/benches/predicate_pushdown.slt

+2-1
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,8 @@ CREATE SINK sink1 AS select * from t1 WITH (
1919
s3.access.key = 'hummockadmin',
2020
s3.secret.key = 'hummockadmin',
2121
commit_checkpoint_interval = 1,
22-
create_table_if_not_exists = 'true'
22+
create_table_if_not_exists = 'true',
23+
partition_by = 'i1'
2324
);
2425

2526
statement ok

e2e_test/iceberg/test_case/iceberg_connection.slt

+1
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ CREATE SINK sink1 from s1 WITH (
2626
create_table_if_not_exists = 'true',
2727
commit_checkpoint_interval = 1,
2828
primary_key = 'i1,i2',
29+
partition_by = 'i1'
2930
);
3031

3132
statement ok

e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt

+2-1
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,8 @@ CREATE SINK sink1 AS select * from mv1 WITH (
3737
s3.access.key = 'hummockadmin',
3838
s3.secret.key = 'hummockadmin',
3939
commit_checkpoint_interval = 1,
40-
create_table_if_not_exists = 'true'
40+
create_table_if_not_exists = 'true',
41+
partition_by = 'i1'
4142
);
4243

4344
statement ok

e2e_test/iceberg/test_case/iceberg_select_empty_table.slt

+2-1
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,8 @@ CREATE SINK sink1 AS select * from mv1 WITH (
2525
s3.access.key = 'hummockadmin',
2626
s3.secret.key = 'hummockadmin',
2727
commit_checkpoint_interval = 1,
28-
create_table_if_not_exists = 'true'
28+
create_table_if_not_exists = 'true',
29+
partition_by = 'i1'
2930
);
3031

3132
statement ok

e2e_test/iceberg/test_case/iceberg_sink_no_partition_append_only_table.slt

+2-1
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,8 @@ CREATE SINK s6 AS select * from mv6 WITH (
3939
s3.region = 'us-east-1',
4040
s3.access.key = 'hummockadmin',
4141
s3.secret.key = 'hummockadmin',
42-
create_table_if_not_exists = 'true'
42+
create_table_if_not_exists = 'true',
43+
partition_by = 'year(v_date)'
4344
);
4445

4546
statement ok

e2e_test/iceberg/test_case/iceberg_source_all_delete.slt

+1
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ CREATE SINK sink1 AS select * from mv1 WITH (
2424
create_table_if_not_exists = 'true',
2525
commit_checkpoint_interval = 3,
2626
primary_key = 'i1,i2',
27+
partition_by = 'i1'
2728
);
2829

2930
statement ok

e2e_test/iceberg/test_case/iceberg_source_equality_delete.slt

+2
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ CREATE SINK sink1 AS select * from mv1 WITH (
2626
s3.secret.key = 'hummockadmin',
2727
create_table_if_not_exists = 'true',
2828
primary_key = 'i1,i2',
29+
partition_by = 'i1'
2930
);
3031

3132
statement ok
@@ -122,6 +123,7 @@ CREATE SINK sink2 AS select * from s2 WITH (
122123
s3.secret.key = 'hummockadmin',
123124
create_table_if_not_exists = 'true',
124125
primary_key = 'i2,i1',
126+
partition_by = 'i1'
125127
);
126128

127129
statement ok

e2e_test/iceberg/test_case/iceberg_source_explain_for_delete.slt

+1
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ CREATE SINK sink1 AS select * from mv1 WITH (
2626
s3.secret.key = 'hummockadmin',
2727
create_table_if_not_exists = 'true',
2828
primary_key = 'i1,i2',
29+
partition_by = 'i1'
2930
);
3031

3132
statement ok

e2e_test/iceberg/test_case/iceberg_source_position_delete.slt

+1
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ CREATE SINK sink1 AS select * from mv1 WITH (
2424
create_table_if_not_exists = 'true',
2525
commit_checkpoint_interval = 3,
2626
primary_key = 'i1,i2',
27+
partition_by = 'i1'
2728
);
2829

2930
statement ok

e2e_test/sink/iceberg_sink.slt

+2-1
Original file line numberDiff line numberDiff line change
@@ -67,7 +67,8 @@ CREATE SINK s7 AS select mv6.v1 as v1, mv6.v2 as v2, mv6.v3 as v3 from mv6 WITH
6767
database.name='demo_db',
6868
table.name='e2e_auto_create_table',
6969
commit_checkpoint_interval = 1,
70-
create_table_if_not_exists = 'true'
70+
create_table_if_not_exists = 'true',
71+
partition_by = 'v1'
7172
);
7273

7374
statement ok

src/connector/src/sink/iceberg/mod.rs

+59-2
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,15 @@ mod prometheus;
1818
use std::collections::{BTreeMap, HashMap};
1919
use std::fmt::Debug;
2020
use std::num::NonZeroU64;
21+
use std::str::FromStr;
2122
use std::sync::Arc;
2223

2324
use anyhow::{anyhow, Context};
2425
use async_trait::async_trait;
2526
use iceberg::arrow::{arrow_schema_to_schema, schema_to_arrow_schema};
26-
use iceberg::spec::{DataFile, SerializedDataFile};
27+
use iceberg::spec::{
28+
DataFile, SerializedDataFile, Transform, UnboundPartitionField, UnboundPartitionSpec,
29+
};
2730
use iceberg::table::Table;
2831
use iceberg::transaction::Transaction;
2932
use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder;
@@ -47,6 +50,7 @@ use itertools::Itertools;
4750
use parquet::file::properties::WriterProperties;
4851
use prometheus::monitored_general_writer::MonitoredGeneralWriterBuilder;
4952
use prometheus::monitored_position_delete_writer::MonitoredPositionDeleteWriterBuilder;
53+
use regex::Regex;
5054
use risingwave_common::array::arrow::arrow_array_iceberg::{Int32Array, RecordBatch};
5155
use risingwave_common::array::arrow::arrow_schema_iceberg::{
5256
self, DataType as ArrowDataType, Field as ArrowField, Schema as ArrowSchema, SchemaRef,
@@ -109,6 +113,13 @@ pub struct IcebergConfig {
109113
#[serde(skip)]
110114
pub java_catalog_props: HashMap<String, String>,
111115

116+
#[serde(
117+
rename = "partition_by",
118+
default,
119+
deserialize_with = "deserialize_optional_string_seq_from_string"
120+
)]
121+
pub partition_by: Option<Vec<String>>,
122+
112123
/// Commit every n(>0) checkpoints, default is 10.
113124
#[serde(default = "default_commit_checkpoint_interval")]
114125
#[serde_as(as = "DisplayFromStr")]
@@ -311,9 +322,53 @@ impl IcebergSink {
311322
}
312323
};
313324

325+
let partition_fields = match &self.config.partition_by {
326+
Some(partition_by) => {
327+
let mut partition_fields =
328+
Vec::<UnboundPartitionField>::with_capacity(partition_by.len());
329+
for partition_field in partition_by {
330+
let re = Regex::new(r"\w+(\(\w+\))?").unwrap();
331+
if !re.is_match(partition_field) {
332+
bail!(format!("Invalid partition field: {}", partition_field))
333+
}
334+
let (_, [field1, field2]) = re.captures(partition_field).unwrap().extract();
335+
let mut func = field1.to_owned();
336+
let mut column = field2.to_owned();
337+
if column.is_empty() {
338+
column.replace_range(.., &func);
339+
func.replace_range(.., "identity");
340+
}
341+
let transform = Transform::from_str(&func).unwrap();
342+
if transform == Transform::Unknown {
343+
bail!(format!("Invalid partition field: {}", partition_field))
344+
}
345+
for (pos, col) in self.param.columns.iter().enumerate() {
346+
if col.name == column {
347+
partition_fields.push(
348+
UnboundPartitionField::builder()
349+
.source_id(pos as i32)
350+
.transform(transform)
351+
.name(column.to_owned())
352+
.build(),
353+
);
354+
break;
355+
}
356+
}
357+
}
358+
partition_fields
359+
}
360+
None => Vec::<UnboundPartitionField>::new(),
361+
};
362+
314363
let table_creation_builder = TableCreation::builder()
315364
.name(self.config.common.table_name.clone())
316-
.schema(iceberg_schema);
365+
.schema(iceberg_schema)
366+
.partition_spec(
367+
UnboundPartitionSpec::builder()
368+
.add_partition_fields(partition_fields)
369+
.unwrap()
370+
.build(),
371+
);
317372

318373
let table_creation = match location {
319374
Some(location) => table_creation_builder.location(location).build(),
@@ -1402,6 +1457,7 @@ mod test {
14021457
("connector", "iceberg"),
14031458
("type", "upsert"),
14041459
("primary_key", "v1"),
1460+
("partition_by", "v1,identity(v2)"),
14051461
("warehouse.path", "s3://iceberg"),
14061462
("s3.endpoint", "http://127.0.0.1:9301"),
14071463
("s3.access.key", "hummockadmin"),
@@ -1445,6 +1501,7 @@ mod test {
14451501
r#type: "upsert".to_owned(),
14461502
force_append_only: false,
14471503
primary_key: Some(vec!["v1".to_owned()]),
1504+
partition_by: Some(vec!["v1".to_owned(), "identity(v2)".to_owned()]),
14481505
java_catalog_props: [("jdbc.user", "admin"), ("jdbc.password", "123456")]
14491506
.into_iter()
14501507
.map(|(k, v)| (k.to_owned(), v.to_owned()))

src/connector/with_options_sink.yaml

+4
Original file line numberDiff line numberDiff line change
@@ -504,6 +504,10 @@ IcebergConfig:
504504
- name: java_catalog_props
505505
field_type: HashMap<String,String>
506506
required: false
507+
- name: partition_by
508+
field_type: Vec<String>
509+
required: false
510+
default: Default::default
507511
- name: commit_checkpoint_interval
508512
field_type: u64
509513
comments: Commit every n(>0) checkpoints, default is 10.

0 commit comments

Comments
 (0)