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): support metadata tables #19873

Draft
wants to merge 1 commit into
base: main
Choose a base branch
from
Draft
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
43 changes: 21 additions & 22 deletions Cargo.lock

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

6 changes: 3 additions & 3 deletions Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -143,9 +143,9 @@ icelake = { git = "https://github.com/risingwavelabs/icelake.git", rev = "0ec44f
"prometheus",
] }
# branch dev-rebase-main-20241030
iceberg = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "53f786fb2141b51d10a173cbcb5595edd5aa52a6" }
iceberg-catalog-rest = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "53f786fb2141b51d10a173cbcb5595edd5aa52a6" }
iceberg-catalog-glue = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "53f786fb2141b51d10a173cbcb5595edd5aa52a6" }
iceberg = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "3ad5df0c3a09ebcaf218b6dc1213fdd5a0d68ccb" }
iceberg-catalog-rest = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "3ad5df0c3a09ebcaf218b6dc1213fdd5a0d68ccb" }
iceberg-catalog-glue = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "3ad5df0c3a09ebcaf218b6dc1213fdd5a0d68ccb" }
opendal = "0.49"
# used only by arrow-udf-flight
arrow-flight = "53"
Expand Down
12 changes: 12 additions & 0 deletions proto/batch_plan.proto
Original file line number Diff line number Diff line change
Expand Up @@ -121,6 +121,17 @@ message MySqlQueryNode {
string query = 7;
}

message IcebergMetadataScanNode {
map<string, string> with_properties = 1;
map<string, secret.SecretRef> secret_refs = 2;
enum IcebergMetadataTableType {
UNSPECIFIED = 0;
SNAPSHOTS = 1;
}
// <https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/MetadataTableType.java#L23-L39>
IcebergMetadataTableType table_type = 3;
}

message ProjectNode {
repeated expr.ExprNode select_list = 1;
}
Expand Down Expand Up @@ -405,6 +416,7 @@ message PlanNode {
IcebergScanNode iceberg_scan = 39;
PostgresQueryNode postgres_query = 40;
MySqlQueryNode mysql_query = 41;
IcebergMetadataScanNode iceberg_metadata_scan = 42;
// The following nodes are used for testing.
bool block_executor = 100;
bool busy_loop_executor = 101;
Expand Down
2 changes: 2 additions & 0 deletions src/batch/executors/src/executor.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ mod generic_exchange;
mod group_top_n;
mod hash_agg;
mod hop_window;
mod iceberg_metadata_scan;
mod iceberg_scan;
mod insert;
mod join;
Expand Down Expand Up @@ -56,6 +57,7 @@ pub use generic_exchange::*;
pub use group_top_n::*;
pub use hash_agg::*;
pub use hop_window::*;
pub use iceberg_metadata_scan::*;
pub use iceberg_scan::*;
pub use insert::*;
pub use join::*;
Expand Down
88 changes: 88 additions & 0 deletions src/batch/executors/src/executor/iceberg_metadata_scan.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
// 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 futures_async_stream::try_stream;
use futures_util::stream::StreamExt;
use risingwave_common::array::arrow::IcebergArrowConvert;
use risingwave_common::array::DataChunk;
use risingwave_common::catalog::Schema;
use risingwave_connector::source::iceberg::IcebergProperties;
use risingwave_connector::source::ConnectorProperties;
use risingwave_connector::WithOptionsSecResolved;
use risingwave_pb::batch_plan::plan_node::NodeBody;

use crate::error::BatchError;
use crate::executor::{BoxedExecutor, BoxedExecutorBuilder, Executor, ExecutorBuilder};

pub struct IcebergMetadataScanExecutor {
schema: Schema,
identity: String,
iceberg_properties: IcebergProperties,
}

impl Executor for IcebergMetadataScanExecutor {
fn schema(&self) -> &risingwave_common::catalog::Schema {
&self.schema
}

fn identity(&self) -> &str {
&self.identity
}

fn execute(self: Box<Self>) -> super::BoxedDataChunkStream {
self.do_execute().boxed()
}
}

impl IcebergMetadataScanExecutor {
#[try_stream(ok = DataChunk, error = BatchError)]
async fn do_execute(self: Box<Self>) {
let table = self.iceberg_properties.load_table_v2().await?;
let snapshot = table.metadata_scan().snapshots()?;
let chunk = IcebergArrowConvert.chunk_from_record_batch(&snapshot)?;
yield chunk;

return Ok(());
}
}

pub struct IcebergMetadataScanExecutorBuilder {}

impl BoxedExecutorBuilder for IcebergMetadataScanExecutorBuilder {
async fn new_boxed_executor(
source: &ExecutorBuilder<'_>,
_inputs: Vec<BoxedExecutor>,
) -> crate::error::Result<BoxedExecutor> {
let node = try_match_expand!(
source.plan_node().get_node_body().unwrap(),
NodeBody::IcebergMetadataScan
)?;

let options_with_secret =
WithOptionsSecResolved::new(node.with_properties.clone(), node.secret_refs.clone());
let iceberg_properties = if let ConnectorProperties::Iceberg(config) =
ConnectorProperties::extract(options_with_secret.clone(), true)?
{
*config
} else {
unreachable!()
};

Ok(Box::new(IcebergMetadataScanExecutor {
iceberg_properties,
identity: source.plan_node().get_identity().clone(),
schema: Schema::new(vec![]),
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The schema should be initialized using iceberg_metadata_table_schema(node.table_type) rather than Schema::new(vec![]) to properly reflect the metadata table's structure. The current empty schema means no columns would be available, which is incorrect.

Spotted by Graphite Reviewer

Is this helpful? React 👍 or 👎 to let us know.

}))
}
}
8 changes: 8 additions & 0 deletions src/common/src/array/arrow/arrow_iceberg.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ pub use super::arrow_53::{
arrow_array, arrow_buffer, arrow_cast, arrow_schema, FromArrow, ToArrow,
};
use crate::array::{Array, ArrayError, ArrayImpl, DataChunk, DataType, DecimalArray};
use crate::catalog::Schema;
use crate::types::StructType;

pub struct IcebergArrowConvert;
Expand All @@ -44,6 +45,13 @@ impl IcebergArrowConvert {
FromArrow::from_record_batch(self, batch)
}

pub fn schema_from_arrow_schema(
&self,
schema: &arrow_schema::Schema,
) -> Result<Schema, ArrayError> {
FromArrow::from_schema(self, schema)
}

pub fn type_from_field(&self, field: &arrow_schema::Field) -> Result<DataType, ArrayError> {
FromArrow::from_field(self, field)
}
Expand Down
10 changes: 10 additions & 0 deletions src/common/src/array/arrow/arrow_impl.rs
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ use itertools::Itertools;
use super::{arrow_array, arrow_buffer, arrow_cast, arrow_schema, ArrowIntervalType};
// Other import should always use the absolute path.
use crate::array::*;
use crate::catalog::{Field, Schema};
use crate::types::*;
use crate::util::iter_util::ZipEqFast;

Expand Down Expand Up @@ -479,6 +480,15 @@ pub trait FromArrow {
Ok(DataChunk::new(columns, batch.num_rows()))
}

fn from_schema(&self, schema: &arrow_schema::Schema) -> Result<Schema, ArrayError> {
let fields = schema
.fields()
.iter()
.map(|f| Ok(Field::new(f.name(), self.from_field(f)?)))
.try_collect::<_, Vec<_>, ArrayError>()?;
Ok(Schema::new(fields))
}

/// Converts Arrow `Fields` to RisingWave `StructType`.
fn from_fields(&self, fields: &arrow_schema::Fields) -> Result<StructType, ArrayError> {
Ok(StructType::new(
Expand Down
13 changes: 13 additions & 0 deletions src/connector/src/source/iceberg/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,15 +21,18 @@ use anyhow::anyhow;
use async_trait::async_trait;
use futures_async_stream::for_await;
use iceberg::expr::Predicate as IcebergPredicate;
use iceberg::metadata_scan::MetadataTable;
use iceberg::scan::FileScanTask;
use iceberg::spec::TableMetadata;
use iceberg::table::Table;
use itertools::Itertools;
pub use parquet_file_handler::*;
use risingwave_common::array::arrow::IcebergArrowConvert;
use risingwave_common::bail;
use risingwave_common::catalog::{Schema, ICEBERG_SEQUENCE_NUM_COLUMN_NAME};
use risingwave_common::types::JsonbVal;
use risingwave_common::util::iter_util::ZipEqFast;
use risingwave_pb::batch_plan::iceberg_metadata_scan_node::IcebergMetadataTableType;
use risingwave_pb::batch_plan::iceberg_scan_node::IcebergScanType;
use serde::{Deserialize, Serialize};

Expand Down Expand Up @@ -494,3 +497,13 @@ impl SplitReader for IcebergFileReader {
unimplemented!()
}
}

pub fn iceberg_metadata_table_schema(table_type: IcebergMetadataTableType) -> Schema {
let arrow_schema = match table_type {
IcebergMetadataTableType::Snapshots => iceberg::metadata_scan::SnapshotsTable::schema(),
_ => unreachable!(),
};
IcebergArrowConvert
.schema_from_arrow_schema(&arrow_schema)
.expect("should be a valid schema")
}
6 changes: 3 additions & 3 deletions src/frontend/src/binder/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -52,9 +52,9 @@ pub use insert::BoundInsert;
use pgwire::pg_server::{Session, SessionId};
pub use query::BoundQuery;
pub use relation::{
BoundBackCteRef, BoundBaseTable, BoundJoin, BoundShare, BoundShareInput, BoundSource,
BoundSystemTable, BoundWatermark, BoundWindowTableFunction, Relation,
ResolveQualifiedNameError, WindowTableFunctionKind,
BoundBackCteRef, BoundBaseTable, BoundIcebergTableFunction, BoundJoin, BoundShare,
BoundShareInput, BoundSource, BoundSystemTable, BoundWatermark, BoundWindowTableFunction,
Relation, ResolveQualifiedNameError, WindowTableFunctionKind,
};
pub use select::{BoundDistinct, BoundSelect};
pub use set_expr::*;
Expand Down
Loading