-
Notifications
You must be signed in to change notification settings - Fork 591
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: support expression to compute iceberg sink partition value #14470
Changes from 4 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. LGTM |
Large diffs are not rendered by default.
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
@@ -0,0 +1,197 @@ | ||||||
// Copyright 2024 RisingWave Labs | ||||||
// | ||||||
// Licensed under the Apache License, Version 2.0 (the "License"); | ||||||
// you may not use this file except in compliance with the License. | ||||||
// You may obtain a copy of the License at | ||||||
// | ||||||
// http://www.apache.org/licenses/LICENSE-2.0 | ||||||
// | ||||||
// Unless required by applicable law or agreed to in writing, software | ||||||
// distributed under the License is distributed on an "AS IS" BASIS, | ||||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||||||
// See the License for the specific language governing permissions and | ||||||
// limitations under the License. | ||||||
|
||||||
//! This module contains the expression for computing the iceberg partition value. | ||||||
//! spec ref: <https://iceberg.apache.org/spec/#partition-transforms> | ||||||
use std::fmt::Formatter; | ||||||
use std::str::FromStr; | ||||||
use std::sync::Arc; | ||||||
|
||||||
use anyhow::anyhow; | ||||||
use arrow_schema::DataType as ArrowDataType; | ||||||
use icelake::types::{ | ||||||
create_transform_function, Any as IcelakeDataType, BoxedTransformFunction, Transform, | ||||||
}; | ||||||
use risingwave_common::array::{ArrayRef, DataChunk}; | ||||||
use risingwave_common::ensure; | ||||||
use risingwave_common::row::OwnedRow; | ||||||
use risingwave_common::types::{DataType, Datum}; | ||||||
use risingwave_expr::expr::BoxedExpression; | ||||||
use risingwave_expr::{build_function, ExprError, Result}; | ||||||
|
||||||
pub struct IcebergTransform { | ||||||
child: BoxedExpression, | ||||||
transform: BoxedTransformFunction, | ||||||
return_type: DataType, | ||||||
} | ||||||
|
||||||
impl std::fmt::Debug for IcebergTransform { | ||||||
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { | ||||||
f.debug_struct("IcebergTransform") | ||||||
.field("child", &self.child) | ||||||
.field("return_type", &self.return_type) | ||||||
.finish() | ||||||
} | ||||||
} | ||||||
|
||||||
#[async_trait::async_trait] | ||||||
impl risingwave_expr::expr::Expression for IcebergTransform { | ||||||
fn return_type(&self) -> DataType { | ||||||
self.return_type.clone() | ||||||
} | ||||||
|
||||||
async fn eval(&self, data_chunk: &DataChunk) -> Result<ArrayRef> { | ||||||
// Get the child array | ||||||
let array = self.child.eval(data_chunk).await?; | ||||||
// Convert to arrow array | ||||||
let arrow_array = array.as_ref().try_into().unwrap(); | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I will convert it as an error in the next PR. |
||||||
// Transform | ||||||
let res_array = self.transform.transform(arrow_array).unwrap(); | ||||||
// Convert back to array ref and return it | ||||||
Ok(Arc::new((&res_array).try_into().unwrap())) | ||||||
} | ||||||
|
||||||
async fn eval_row(&self, _row: &OwnedRow) -> Result<Datum> { | ||||||
Err(ExprError::Internal(anyhow!( | ||||||
"eval_row in iceberg_transform is not supported yet" | ||||||
))) | ||||||
} | ||||||
} | ||||||
|
||||||
#[build_function("iceberg_transform(varchar, any) -> any", type_infer = "panic")] | ||||||
fn build(return_type: DataType, mut children: Vec<BoxedExpression>) -> Result<BoxedExpression> { | ||||||
let transform_type = { | ||||||
let datum = children[0].eval_const()?.unwrap(); | ||||||
let str = datum.as_utf8(); | ||||||
Transform::from_str(str).map_err(|_| ExprError::InvalidParam { | ||||||
name: "transform type in icberg_transform", | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
|
||||||
reason: format!("Fail to parse {str} as iceberg transform type").into(), | ||||||
})? | ||||||
}; | ||||||
|
||||||
// For Identity and Void transform, we will use `InputRef` and const null in frontend, | ||||||
// so it should not reach here. | ||||||
assert!(!matches!( | ||||||
transform_type, | ||||||
Transform::Identity | Transform::Void | ||||||
)); | ||||||
|
||||||
// Check type: | ||||||
// 1. input type can be transform successfully | ||||||
// 2. return type is the same as the result type | ||||||
let input_type = IcelakeDataType::try_from(ArrowDataType::try_from(children[1].return_type())?) | ||||||
.map_err(|err| ExprError::InvalidParam { | ||||||
name: "input type in iceberg_transform", | ||||||
reason: format!("Failed to convert input type to icelake type, got error: {err}",) | ||||||
.into(), | ||||||
})?; | ||||||
let expect_res_type = transform_type.result_type(&input_type).map_err( | ||||||
|err| ExprError::InvalidParam { | ||||||
name: "input type in iceberg_transform", | ||||||
reason: format!( | ||||||
"Failed to get result type for transform type {:?} and input type {:?}, got error: {}", | ||||||
transform_type, input_type, err | ||||||
) | ||||||
.into() | ||||||
})?; | ||||||
let actual_res_type = IcelakeDataType::try_from(ArrowDataType::try_from(return_type.clone())?) | ||||||
.map_err(|err| ExprError::InvalidParam { | ||||||
name: "return type in iceberg_transform", | ||||||
reason: format!("Failed to convert return type to icelake type, got error: {err}",) | ||||||
.into(), | ||||||
})?; | ||||||
ensure!( | ||||||
expect_res_type == actual_res_type, | ||||||
ExprError::InvalidParam { | ||||||
name: "return type in iceberg_transform", | ||||||
reason: format!( | ||||||
"Expect return type {:?} but got {:?}", | ||||||
expect_res_type, actual_res_type | ||||||
) | ||||||
.into() | ||||||
} | ||||||
); | ||||||
|
||||||
Ok(Box::new(IcebergTransform { | ||||||
child: children.remove(1), | ||||||
transform: create_transform_function(&transform_type) | ||||||
.map_err(|err| ExprError::Internal(err.into()))?, | ||||||
return_type, | ||||||
})) | ||||||
} | ||||||
|
||||||
#[cfg(test)] | ||||||
mod test { | ||||||
use risingwave_common::array::{DataChunk, DataChunkTestExt}; | ||||||
use risingwave_expr::expr::build_from_pretty; | ||||||
|
||||||
#[tokio::test] | ||||||
async fn test_bucket() { | ||||||
let (input, expected) = DataChunk::from_pretty( | ||||||
"i i | ||||||
34 1373", | ||||||
) | ||||||
.split_column_at(1); | ||||||
let expr = build_from_pretty("(iceberg_transform:int4 bucket[2017]:varchar $0:int)"); | ||||||
let res = expr.eval(&input).await.unwrap(); | ||||||
assert_eq!(res, *expected.column_at(0)); | ||||||
} | ||||||
|
||||||
#[tokio::test] | ||||||
async fn test_truncate() { | ||||||
let (input, expected) = DataChunk::from_pretty( | ||||||
"T T | ||||||
iceberg ice | ||||||
risingwave ris | ||||||
delta del", | ||||||
) | ||||||
.split_column_at(1); | ||||||
let expr = build_from_pretty("(iceberg_transform:varchar truncate[3]:varchar $0:varchar)"); | ||||||
let res = expr.eval(&input).await.unwrap(); | ||||||
assert_eq!(res, *expected.column_at(0)); | ||||||
} | ||||||
|
||||||
#[tokio::test] | ||||||
async fn test_year_month_day_hour() { | ||||||
let (input, expected) = DataChunk::from_pretty( | ||||||
"TZ i i i i | ||||||
1970-01-01T00:00:00.000000000+00:00 0 0 0 0 | ||||||
1971-02-01T01:00:00.000000000+00:00 1 13 396 9505 | ||||||
1972-03-01T02:00:00.000000000+00:00 2 26 790 18962 | ||||||
1970-05-01T06:00:00.000000000+00:00 0 4 120 2886 | ||||||
1970-06-01T07:00:00.000000000+00:00 0 5 151 3631", | ||||||
) | ||||||
.split_column_at(1); | ||||||
|
||||||
// year | ||||||
let expr = build_from_pretty("(iceberg_transform:int4 year:varchar $0:timestamptz)"); | ||||||
let res = expr.eval(&input).await.unwrap(); | ||||||
assert_eq!(res, *expected.column_at(0)); | ||||||
|
||||||
// month | ||||||
let expr = build_from_pretty("(iceberg_transform:int4 month:varchar $0:timestamptz)"); | ||||||
let res = expr.eval(&input).await.unwrap(); | ||||||
assert_eq!(res, *expected.column_at(1)); | ||||||
|
||||||
// day | ||||||
let expr = build_from_pretty("(iceberg_transform:int4 day:varchar $0:timestamptz)"); | ||||||
let res = expr.eval(&input).await.unwrap(); | ||||||
assert_eq!(res, *expected.column_at(2)); | ||||||
|
||||||
// hour | ||||||
let expr = build_from_pretty("(iceberg_transform:int4 hour:varchar $0:timestamptz)"); | ||||||
let res = expr.eval(&input).await.unwrap(); | ||||||
assert_eq!(res, *expected.column_at(3)); | ||||||
} | ||||||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,16 @@ | ||
// Copyright 2024 RisingWave Labs | ||
// | ||
// Licensed under the Apache License, Version 2.0 (the "License"); | ||
// you may not use this file except in compliance with the License. | ||
// You may obtain a copy of the License at | ||
// | ||
// http://www.apache.org/licenses/LICENSE-2.0 | ||
// | ||
// Unless required by applicable law or agreed to in writing, software | ||
// distributed under the License is distributed on an "AS IS" BASIS, | ||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
// See the License for the specific language governing permissions and | ||
// limitations under the License. | ||
|
||
//! This module contain the expression for external system. Such as partition compute expression for external sink. | ||
pub mod iceberg; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmmm, why Cargo.lock has so many updates? Especially why
syn
,serde
is updated?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think it's because
avro
specified the minor number (and uses dependabot) and thus forces downstream to update.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not a blocker. But want to take a look at the CHANGELOG of
syn
,serde
,thiserror