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(iceberg): add support for creating new iceberg tables with partition keys #20340

Open
wants to merge 3 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
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
3 changes: 2 additions & 1 deletion e2e_test/iceberg/benches/predicate_pushdown.slt
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,8 @@ CREATE SINK sink1 AS select * from t1 WITH (
s3.access.key = 'hummockadmin',
s3.secret.key = 'hummockadmin',
commit_checkpoint_interval = 1,
create_table_if_not_exists = 'true'
create_table_if_not_exists = 'true',
partition_by = 'i1'
);

statement ok
Expand Down
1 change: 1 addition & 0 deletions e2e_test/iceberg/test_case/iceberg_connection.slt
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ CREATE SINK sink1 from s1 WITH (
create_table_if_not_exists = 'true',
commit_checkpoint_interval = 1,
primary_key = 'i1,i2',
partition_by = 'i1'
);

statement ok
Expand Down
3 changes: 2 additions & 1 deletion e2e_test/iceberg/test_case/iceberg_predicate_pushdown.slt
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,8 @@ CREATE SINK sink1 AS select * from mv1 WITH (
s3.access.key = 'hummockadmin',
s3.secret.key = 'hummockadmin',
commit_checkpoint_interval = 1,
create_table_if_not_exists = 'true'
create_table_if_not_exists = 'true',
partition_by = 'i1'
);

statement ok
Expand Down
3 changes: 2 additions & 1 deletion e2e_test/iceberg/test_case/iceberg_select_empty_table.slt
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,8 @@ CREATE SINK sink1 AS select * from mv1 WITH (
s3.access.key = 'hummockadmin',
s3.secret.key = 'hummockadmin',
commit_checkpoint_interval = 1,
create_table_if_not_exists = 'true'
create_table_if_not_exists = 'true',
partition_by = 'i1'
);

statement ok
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,8 @@ CREATE SINK s6 AS select * from mv6 WITH (
s3.region = 'us-east-1',
s3.access.key = 'hummockadmin',
s3.secret.key = 'hummockadmin',
create_table_if_not_exists = 'true'
create_table_if_not_exists = 'true',
partition_by = 'year(v_date)'
);

statement ok
Expand Down
1 change: 1 addition & 0 deletions e2e_test/iceberg/test_case/iceberg_source_all_delete.slt
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ CREATE SINK sink1 AS select * from mv1 WITH (
create_table_if_not_exists = 'true',
commit_checkpoint_interval = 3,
primary_key = 'i1,i2',
partition_by = 'i1'
);

statement ok
Expand Down
2 changes: 2 additions & 0 deletions e2e_test/iceberg/test_case/iceberg_source_equality_delete.slt
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ CREATE SINK sink1 AS select * from mv1 WITH (
s3.secret.key = 'hummockadmin',
create_table_if_not_exists = 'true',
primary_key = 'i1,i2',
partition_by = 'i1'
);

statement ok
Expand Down Expand Up @@ -122,6 +123,7 @@ CREATE SINK sink2 AS select * from s2 WITH (
s3.secret.key = 'hummockadmin',
create_table_if_not_exists = 'true',
primary_key = 'i2,i1',
partition_by = 'i1'
);

statement ok
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ CREATE SINK sink1 AS select * from mv1 WITH (
s3.secret.key = 'hummockadmin',
create_table_if_not_exists = 'true',
primary_key = 'i1,i2',
partition_by = 'i1'
);

statement ok
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ CREATE SINK sink1 AS select * from mv1 WITH (
create_table_if_not_exists = 'true',
commit_checkpoint_interval = 3,
primary_key = 'i1,i2',
partition_by = 'i1'
);

statement ok
Expand Down
3 changes: 2 additions & 1 deletion e2e_test/sink/iceberg_sink.slt
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,8 @@ CREATE SINK s7 AS select mv6.v1 as v1, mv6.v2 as v2, mv6.v3 as v3 from mv6 WITH
database.name='demo_db',
table.name='e2e_auto_create_table',
commit_checkpoint_interval = 1,
create_table_if_not_exists = 'true'
create_table_if_not_exists = 'true',
partition_by = 'v1'
);

statement ok
Expand Down
68 changes: 66 additions & 2 deletions src/connector/src/sink/iceberg/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,15 @@ mod prometheus;
use std::collections::{BTreeMap, HashMap};
use std::fmt::Debug;
use std::num::NonZeroU64;
use std::str::FromStr;
use std::sync::Arc;

use anyhow::{anyhow, Context};
use async_trait::async_trait;
use iceberg::arrow::{arrow_schema_to_schema, schema_to_arrow_schema};
use iceberg::spec::{DataFile, SerializedDataFile};
use iceberg::spec::{
DataFile, SerializedDataFile, Transform, UnboundPartitionField, UnboundPartitionSpec,
};
use iceberg::table::Table;
use iceberg::transaction::Transaction;
use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder;
Expand All @@ -47,6 +50,7 @@ use itertools::Itertools;
use parquet::file::properties::WriterProperties;
use prometheus::monitored_general_writer::MonitoredGeneralWriterBuilder;
use prometheus::monitored_position_delete_writer::MonitoredPositionDeleteWriterBuilder;
use regex::Regex;
use risingwave_common::array::arrow::arrow_array_iceberg::{Int32Array, RecordBatch};
use risingwave_common::array::arrow::arrow_schema_iceberg::{
self, DataType as ArrowDataType, Field as ArrowField, Schema as ArrowSchema, SchemaRef,
Expand Down Expand Up @@ -109,6 +113,9 @@ pub struct IcebergConfig {
#[serde(skip)]
pub java_catalog_props: HashMap<String, String>,

#[serde(default)]
pub partition_by: Option<String>,

/// Commit every n(>0) checkpoints, default is 10.
#[serde(default = "default_commit_checkpoint_interval")]
#[serde_as(as = "DisplayFromStr")]
Expand Down Expand Up @@ -311,9 +318,64 @@ impl IcebergSink {
}
};

let partition_fields = match &self.config.partition_by {
Some(partition_field) => {
let mut partition_fields = Vec::<UnboundPartitionField>::new();
// captures column, transform(column), transform(n,column), transform(n, column)
let re = Regex::new(
r"(?<transform>\w+)(\(((?<n>\d+)?(?:,|(,\s)))?(?<field>\w+)\))?",
)
.unwrap();
if !re.is_match(partition_field) {
bail!(format!("Invalid partition fields: {}", partition_field))
}
let caps = re.captures_iter(partition_field);
for mat in caps {
let (column, transform) = if mat["n"].is_empty() && mat["field"].is_empty()
{
(&mat["func"], Transform::Identity)
} else {
let mut func = mat["func"].to_owned();
let n = &mat["n"];
if func == "bucket" || func == "truncate" {
func = format!("{func}({n})");
}
(&mat["field"], Transform::from_str(&func).unwrap())
};

let mut exists = false;
for (pos, col) in self.param.columns.iter().enumerate() {
if col.name == column {
partition_fields.push(
UnboundPartitionField::builder()
.source_id(pos as i32)
.transform(transform)
.name(column.to_owned())
.build(),
);
exists = true;
break;
}
}
// safety check
if !exists {
bail!(format!("Invalid partition column: {}", column))
}
}
partition_fields
}
None => Vec::<UnboundPartitionField>::new(),
};

let table_creation_builder = TableCreation::builder()
.name(self.config.common.table_name.clone())
.schema(iceberg_schema);
.schema(iceberg_schema)
.partition_spec(
UnboundPartitionSpec::builder()
.add_partition_fields(partition_fields)
.unwrap()
.build(),
);

let table_creation = match location {
Some(location) => table_creation_builder.location(location).build(),
Expand Down Expand Up @@ -1402,6 +1464,7 @@ mod test {
("connector", "iceberg"),
("type", "upsert"),
("primary_key", "v1"),
("partition_by", "v1, identity(v1), truncate(4,v2), bucket(5,v1), year(v3), month(v4), day(v5), hour(v6), void(v1)"),
("warehouse.path", "s3://iceberg"),
("s3.endpoint", "http://127.0.0.1:9301"),
("s3.access.key", "hummockadmin"),
Expand Down Expand Up @@ -1446,6 +1509,7 @@ mod test {
r#type: "upsert".to_owned(),
force_append_only: false,
primary_key: Some(vec!["v1".to_owned()]),
partition_by: Some("v1, identity(v1), truncate(4,v2), bucket(5,v1), year(v3), month(v4), day(v5), hour(v6), void(v1)".to_owned()),
java_catalog_props: [("jdbc.user", "admin"), ("jdbc.password", "123456")]
.into_iter()
.map(|(k, v)| (k.to_owned(), v.to_owned()))
Expand Down
4 changes: 4 additions & 0 deletions src/connector/with_options_sink.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -511,6 +511,10 @@ IcebergConfig:
- name: java_catalog_props
field_type: HashMap<String,String>
required: false
- name: partition_by
field_type: String
required: false
default: Default::default
- name: commit_checkpoint_interval
field_type: u64
comments: Commit every n(>0) checkpoints, default is 10.
Expand Down
Loading