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: support precompute partition for iceberg #14710

Merged
merged 8 commits into from
Feb 1, 2024
Merged
Show file tree
Hide file tree
Changes from 6 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
5 changes: 4 additions & 1 deletion Cargo.lock

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

2 changes: 1 addition & 1 deletion Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ tonic = { package = "madsim-tonic", version = "0.4.1" }
tonic-build = { package = "madsim-tonic-build", version = "0.4.2" }
otlp-embedded = { git = "https://github.com/risingwavelabs/otlp-embedded", rev = "58c1f003484449d7c6dd693b348bf19dd44889cb" }
prost = { version = "0.12" }
icelake = { git = "https://github.com/icelake-io/icelake", rev = "32c0bbf242f5c47b1e743f10577012fe7436c770", features = [
icelake = { git = "https://github.com/icelake-io/icelake", rev = "3f61f900d6914d4a28c00c2af6374a4dda6d95d4", features = [
"prometheus",
] }
arrow-array = "50"
Expand Down
9 changes: 5 additions & 4 deletions e2e_test/iceberg/start_spark_connect_server.sh
Original file line number Diff line number Diff line change
Expand Up @@ -8,9 +8,10 @@ PACKAGES="$PACKAGES,org.apache.spark:spark-connect_2.12:$SPARK_VERSION"

SPARK_FILE="spark-${SPARK_VERSION}-bin-hadoop3.tgz"


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

./spark-${SPARK_VERSION}-bin-hadoop3/sbin/start-connect-server.sh --packages $PACKAGES \
--master local[3] \
Expand All @@ -31,4 +32,4 @@ while ! nc -z localhost 15002; do
sleep 1 # wait for 1/10 of the second before check again
done

echo "Spark connect server launched"
echo "Spark connect server launched"
2 changes: 1 addition & 1 deletion e2e_test/iceberg/test_case/partition_append_only.toml
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@ init_sqls = [
v_timestamp timestamp,
v_ts_ntz timestamp_ntz
)
PARTITIONED BY (v_int,v_long,v_float,v_double,v_varchar,v_bool,v_date,v_timestamp,v_ts_ntz)
PARTITIONED BY (v_int,bucket(10,v_long),truncate(30,v_long),years(v_date),months(v_timestamp),days(v_ts_ntz))
TBLPROPERTIES ('format-version'='2');
'''
]
Expand Down
2 changes: 1 addition & 1 deletion e2e_test/iceberg/test_case/partition_upsert.toml
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ init_sqls = [
v2 long,
v3 string
) USING iceberg
PARTITIONED BY (v1,v2)
PARTITIONED BY (v1,v2,truncate(2,v3))
TBLPROPERTIES ('format-version'='2');
'''
]
Expand Down
1 change: 1 addition & 0 deletions proto/stream_plan.proto
Original file line number Diff line number Diff line change
Expand Up @@ -223,6 +223,7 @@ message SinkDesc {
string sink_from_name = 12;
catalog.SinkFormatDesc format_desc = 13;
optional uint32 target_table = 14;
optional uint64 extra_partition_col_idx = 15;
liurenjie1024 marked this conversation as resolved.
Show resolved Hide resolved
}

enum SinkLogStoreType {
Expand Down
73 changes: 73 additions & 0 deletions src/common/src/array/arrow/arrow_iceberg.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
// 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.

use std::sync::Arc;
ZENOTME marked this conversation as resolved.
Show resolved Hide resolved

use arrow_array::{ArrayRef, StructArray};
use arrow_schema::DataType;
use itertools::Itertools;

use crate::array::{ArrayError, DataChunk};
use crate::util::iter_util::ZipEqFast;

/// Converts RisingWave array to Arrow array with the schema.
/// The behavior is specified for iceberg:
/// For different struct type, try to use fields in schema to cast.
pub fn to_iceberg_record_batch_with_schema(
schema: arrow_schema::SchemaRef,
chunk: &DataChunk,
) -> Result<arrow_array::RecordBatch, ArrayError> {
if !chunk.is_compacted() {
let c = chunk.clone();
return to_iceberg_record_batch_with_schema(schema, &c.compact());
}
let columns: Vec<_> = chunk
.columns()
.iter()
.zip_eq_fast(schema.fields().iter())
.map(|(column, field)| {
let column: arrow_array::ArrayRef = column.as_ref().try_into()?;
if column.data_type() == field.data_type() {
Ok(column)
} else if let DataType::Struct(actual) = column.data_type()
&& let DataType::Struct(expect) = field.data_type()
{
// Special case for iceberg
if actual.len() != expect.len() {
return Err(ArrayError::to_arrow(format!(
"Struct field count mismatch, expect {}, actual {}",
expect.len(),
actual.len()
)));
}
let column = column
.as_any()
.downcast_ref::<arrow_array::StructArray>()
.unwrap()
.clone();
let (_, struct_columns, nullable) = column.into_parts();
Ok(Arc::new(
StructArray::try_new(expect.clone(), struct_columns, nullable)
.map_err(ArrayError::from_arrow)?,
) as ArrayRef)
} else {
arrow_cast::cast(&column, field.data_type()).map_err(ArrayError::from_arrow)
}
})
.try_collect::<_, _, ArrayError>()?;

let opts = arrow_array::RecordBatchOptions::default().with_row_count(Some(chunk.capacity()));
arrow_array::RecordBatch::try_new_with_options(schema, columns, &opts)
.map_err(ArrayError::to_arrow)
}
9 changes: 8 additions & 1 deletion src/common/src/array/arrow/arrow_impl.rs
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,7 @@ macro_rules! converts_generic {
fn try_from(array: &ArrayImpl) -> Result<Self, Self::Error> {
match array {
$($ArrayImplPattern(a) => Ok(Arc::new(<$ArrowType>::try_from(a)?)),)*
ArrayImpl::Timestamptz(a) => Ok(Arc::new(arrow_array::TimestampMicrosecondArray::try_from(a)?. with_timezone_utc())),
_ => todo!("unsupported array"),
}
}
Expand All @@ -152,6 +153,13 @@ macro_rules! converts_generic {
.unwrap()
.try_into()?,
)),)*
Timestamp(Microsecond, Some(_)) => Ok(ArrayImpl::Timestamptz(
array
.as_any()
.downcast_ref::<arrow_array::TimestampMicrosecondArray>()
.unwrap()
.try_into()?,
)),
t => Err(ArrayError::from_arrow(format!("unsupported data type: {t:?}"))),
}
}
Expand All @@ -173,7 +181,6 @@ converts_generic! {
{ arrow_array::Decimal256Array, Decimal256(_, _), ArrayImpl::Int256 },
{ arrow_array::Date32Array, Date32, ArrayImpl::Date },
{ arrow_array::TimestampMicrosecondArray, Timestamp(Microsecond, None), ArrayImpl::Timestamp },
{ arrow_array::TimestampMicrosecondArray, Timestamp(Microsecond, Some(_)), ArrayImpl::Timestamptz },
{ arrow_array::Time64MicrosecondArray, Time64(Microsecond), ArrayImpl::Time },
{ arrow_array::IntervalMonthDayNanoArray, Interval(MonthDayNano), ArrayImpl::Interval },
{ arrow_array::StructArray, Struct(_), ArrayImpl::Struct },
Expand Down
2 changes: 2 additions & 0 deletions src/common/src/array/arrow/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@

mod arrow_default;
mod arrow_deltalake;
mod arrow_iceberg;

pub use arrow_default::to_record_batch_with_schema;
pub use arrow_deltalake::to_deltalake_record_batch_with_schema;
pub use arrow_iceberg::to_iceberg_record_batch_with_schema;
5 changes: 4 additions & 1 deletion src/common/src/array/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,10 @@
//! `Array` defines all in-memory representations of vectorized execution framework.

mod arrow;
pub use arrow::{to_deltalake_record_batch_with_schema, to_record_batch_with_schema};
pub use arrow::{
to_deltalake_record_batch_with_schema, to_iceberg_record_batch_with_schema,
to_record_batch_with_schema,
};
mod bool_array;
pub mod bytes_array;
mod chrono_array;
Expand Down
2 changes: 2 additions & 0 deletions src/connector/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,9 @@ apache-avro = { git = "https://github.com/risingwavelabs/avro", rev = "d0846a16c
"xz",
] }
arrow-array = { workspace = true }
arrow-row = { workspace = true }
arrow-schema = { workspace = true }
arrow-select = { workspace = true }
async-nats = "0.33"
async-trait = "0.1"
auto_enums = { version = "0.8", features = ["futures03"] }
Expand Down
4 changes: 4 additions & 0 deletions src/connector/src/sink/catalog/desc.rs
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,9 @@ pub struct SinkDesc {

/// Id of the target table for sink into table.
pub target_table: Option<TableId>,

/// See the same name field in `SinkWriterParam`.
pub extra_partition_col_idx: Option<usize>,
liurenjie1024 marked this conversation as resolved.
Show resolved Hide resolved
}

impl SinkDesc {
Expand Down Expand Up @@ -123,6 +126,7 @@ impl SinkDesc {
db_name: self.db_name.clone(),
sink_from_name: self.sink_from_name.clone(),
target_table: self.target_table.map(|table_id| table_id.table_id()),
extra_partition_col_idx: self.extra_partition_col_idx.map(|idx| idx as u64),
}
}
}
155 changes: 155 additions & 0 deletions src/connector/src/sink/iceberg/mock_catalog.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,155 @@
// 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.

use std::collections::HashMap;
liurenjie1024 marked this conversation as resolved.
Show resolved Hide resolved
use std::sync::Arc;

use async_trait::async_trait;
use icelake::catalog::{Catalog, UpdateTable};
use icelake::types::{Field, PartitionField, Schema, Struct, TableMetadata};
use icelake::{Table, TableIdentifier};
use opendal::services::Memory;
use opendal::Operator;

/// A mock catalog for iceberg used for plan test.
pub struct MockCatalog;

#[async_trait]
impl Catalog for MockCatalog {
fn name(&self) -> &str {
"mock"
}

async fn load_table(self: Arc<Self>, table_name: &TableIdentifier) -> icelake::Result<Table> {
// A mock table for test
let table = Table::builder_from_catalog(
{
let mut builder = Memory::default();
builder.root("/tmp");
Operator::new(builder).unwrap().finish()
},
self.clone(),
TableMetadata {
format_version: icelake::types::TableFormatVersion::V2,
table_uuid: "1".to_string(),
location: "1".to_string(),
last_sequence_number: 1,
last_updated_ms: 1,
last_column_id: 1,
schemas: vec![Schema::new(
1,
None,
Struct::new(vec![
Field::required(
1,
"v1",
icelake::types::Any::Primitive(icelake::types::Primitive::Int),
)
.into(),
Field::required(
2,
"v2",
icelake::types::Any::Primitive(icelake::types::Primitive::Timestamp),
)
.into(),
Field::required(
3,
"v3",
icelake::types::Any::Primitive(icelake::types::Primitive::Timestampz),
)
.into(),
]),
)],
current_schema_id: 1,
partition_specs: vec![icelake::types::PartitionSpec {
spec_id: 1,
fields: vec![
PartitionField {
source_column_id: 1,
partition_field_id: 4,
transform: icelake::types::Transform::Identity,
name: "f1".to_string(),
},
PartitionField {
source_column_id: 1,
partition_field_id: 5,
transform: icelake::types::Transform::Bucket(1),
name: "f2".to_string(),
},
PartitionField {
source_column_id: 1,
partition_field_id: 6,
transform: icelake::types::Transform::Truncate(1),
name: "f3".to_string(),
},
PartitionField {
source_column_id: 2,
partition_field_id: 7,
transform: icelake::types::Transform::Year,
name: "f4".to_string(),
},
PartitionField {
source_column_id: 2,
partition_field_id: 8,
transform: icelake::types::Transform::Month,
name: "f5".to_string(),
},
PartitionField {
source_column_id: 3,
partition_field_id: 9,
transform: icelake::types::Transform::Day,
name: "f6".to_string(),
},
PartitionField {
source_column_id: 3,
partition_field_id: 10,
transform: icelake::types::Transform::Hour,
name: "f7".to_string(),
},
PartitionField {
source_column_id: 1,
partition_field_id: 11,
transform: icelake::types::Transform::Void,
name: "f8".to_string(),
},
PartitionField {
source_column_id: 2,
partition_field_id: 12,
transform: icelake::types::Transform::Void,
name: "f9".to_string(),
},
],
}],
default_spec_id: 1,
last_partition_id: 1,
properties: None,
current_snapshot_id: None,
snapshots: None,
snapshot_log: None,
metadata_log: None,
sort_orders: vec![],
default_sort_order_id: 0,
refs: HashMap::new(),
},
table_name.clone(),
)
.build()
.unwrap();
Ok(table)
}

async fn update_table(self: Arc<Self>, _update_table: &UpdateTable) -> icelake::Result<Table> {
unimplemented!()
}
}
Loading
Loading