-
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(batch): support iceberg scan executor #14915
Merged
+258
−51
Merged
Changes from 1 commit
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
7888b8f
support iceberg scan executor
chenzl25 5e4a47a
refactor
chenzl25 11fdfcd
Merge branch 'main' into dylan/support_iceberg_scan_executor
chenzl25 8f0681b
fmt
chenzl25 52de44c
fix doc
chenzl25 3397250
Merge branch 'main' into dylan/support_iceberg_scan_executor
chenzl25 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,185 @@ | ||
// 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::hash::{DefaultHasher, Hash, Hasher}; | ||
use std::sync::Arc; | ||
|
||
use anyhow::anyhow; | ||
use arrow_array::RecordBatch; | ||
use futures_async_stream::try_stream; | ||
use futures_util::stream::StreamExt; | ||
use icelake::io::{FileScan, TableScan}; | ||
use icelake::TableIdentifier; | ||
use risingwave_common::catalog::Schema; | ||
use risingwave_connector::sink::iceberg::IcebergConfig; | ||
|
||
use crate::error::BatchError; | ||
use crate::executor::{DataChunk, Executor}; | ||
|
||
/// Create a iceberg scan executor. | ||
/// | ||
/// # Examples | ||
/// | ||
/// ``` | ||
/// use futures_async_stream::for_await; | ||
/// use risingwave_common::catalog::{Field, Schema}; | ||
/// use risingwave_common::types::DataType; | ||
/// use risingwave_connector::sink::iceberg::IcebergConfig; | ||
/// | ||
/// use crate::executor::iceberg_scan::{FileSelector, IcebergScanExecutor}; | ||
/// use crate::executor::Executor; | ||
/// | ||
/// #[tokio::test] | ||
/// async fn test_iceberg_scan() { | ||
/// let iceberg_scan_executor = IcebergScanExecutor { | ||
/// database_name: "demo_db".into(), | ||
/// table_name: "demo_table".into(), | ||
/// file_selector: FileSelector::select_all(), | ||
/// iceberg_config: IcebergConfig { | ||
/// database_name: "demo_db".into(), | ||
/// table_name: "demo_table".into(), | ||
/// catalog_type: Some("storage".into()), | ||
/// path: "s3a://hummock001/".into(), | ||
/// endpoint: Some("http://127.0.0.1:9301".into()), | ||
/// access_key: "hummockadmin".into(), | ||
/// secret_key: "hummockadmin".into(), | ||
/// region: Some("us-east-1".into()), | ||
/// ..Default::default() | ||
/// }, | ||
/// snapshot_id: None, | ||
/// batch_size: 1024, | ||
/// schema: Schema::new(vec![ | ||
/// Field::with_name(DataType::Int64, "seq_id"), | ||
/// Field::with_name(DataType::Int64, "user_id"), | ||
/// Field::with_name(DataType::Varchar, "user_name"), | ||
/// ]), | ||
/// identity: "iceberg_scan".into(), | ||
/// }; | ||
/// | ||
/// let stream = Box::new(iceberg_scan_executor).execute(); | ||
/// #[for_await] | ||
/// for chunk in stream { | ||
/// let chunk = chunk.unwrap(); | ||
/// println!("{:?}", chunk); | ||
/// } | ||
/// } | ||
/// ``` | ||
|
||
pub struct IcebergScanExecutor { | ||
database_name: String, | ||
table_name: String, | ||
iceberg_config: IcebergConfig, | ||
snapshot_id: Option<i64>, | ||
file_selector: FileSelector, | ||
batch_size: usize, | ||
|
||
schema: Schema, | ||
identity: String, | ||
} | ||
|
||
impl Executor for IcebergScanExecutor { | ||
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 IcebergScanExecutor { | ||
#[try_stream(ok = DataChunk, error = BatchError)] | ||
async fn do_execute(self: Box<Self>) { | ||
let catalog = self.iceberg_config.create_catalog().await?; | ||
|
||
let table_ident = TableIdentifier::new(vec![self.database_name, self.table_name]).unwrap(); | ||
let table = catalog | ||
.load_table(&table_ident) | ||
.await | ||
.map_err(BatchError::Iceberg)?; | ||
|
||
let table_scan: TableScan = table | ||
.new_scan_builder() | ||
.with_snapshot_id( | ||
self.snapshot_id | ||
.unwrap_or_else(|| table.current_table_metadata().current_snapshot_id.unwrap()), | ||
) | ||
.with_batch_size(self.batch_size) | ||
.with_column_names(self.schema.names()) | ||
.build() | ||
.map_err(|e| BatchError::Internal(anyhow!(e)))?; | ||
let file_scan_stream: icelake::io::FileScanStream = | ||
table_scan.scan(&table).await.map_err(BatchError::Iceberg)?; | ||
|
||
#[for_await] | ||
for file_scan in file_scan_stream { | ||
let file_scan: FileScan = file_scan.map_err(BatchError::Iceberg)?; | ||
if !self.file_selector.select(file_scan.path()) { | ||
continue; | ||
} | ||
let record_batch_stream = file_scan.scan().await.map_err(BatchError::Iceberg)?; | ||
|
||
#[for_await] | ||
for record_batch in record_batch_stream { | ||
let record_batch: RecordBatch = record_batch.map_err(BatchError::Iceberg)?; | ||
let chunk = Self::record_batch_to_chunk(record_batch)?; | ||
debug_assert_eq!(chunk.data_types(), self.schema.data_types()); | ||
yield chunk; | ||
} | ||
} | ||
} | ||
|
||
fn record_batch_to_chunk(record_batch: RecordBatch) -> Result<DataChunk, BatchError> { | ||
let mut columns = Vec::with_capacity(record_batch.num_columns()); | ||
for array in record_batch.columns() { | ||
let column = Arc::new(array.try_into()?); | ||
columns.push(column); | ||
} | ||
Ok(DataChunk::new(columns, record_batch.num_rows())) | ||
} | ||
} | ||
|
||
pub enum FileSelector { | ||
// File paths to be scanned by this executor are specified. | ||
FileList(Vec<String>), | ||
// Data files to be scanned by this executor could be calculated by Hash(file_path) % num_tasks == task_id. | ||
// task_id, num_tasks | ||
Hash(usize, usize), | ||
} | ||
|
||
impl FileSelector { | ||
fn select_all() -> Self { | ||
FileSelector::Hash(0, 1) | ||
} | ||
|
||
fn select(&self, path: &str) -> bool { | ||
match self { | ||
FileSelector::FileList(paths) => paths.contains(&path.to_string()), | ||
FileSelector::Hash(task_id, num_tasks) => { | ||
let hash = Self::hash_str_to_usize(path); | ||
hash % num_tasks == *task_id | ||
} | ||
} | ||
} | ||
|
||
fn hash_str_to_usize(s: &str) -> usize { | ||
let mut hasher = DefaultHasher::new(); | ||
s.hash(&mut hasher); | ||
hasher.finish() as usize | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Usually we do this in planning phase, so that we only need to read metadata for only once, and the scan executor only needs to contains file names.
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.
wondering if we can do the same as long as setting
snapshot_id
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.
However, icelake currently doesn't provide a way to read Files directly. If we have a better API later, we can avoid such planning in the future.
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.
Yes, in fact we should set
snapshot_id
, otherwise different tasks may see different snapshots.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.
cc @ZENOTME Could you help to add this api in icelake?
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 would suggest to pending this to wait for icelake's api support, WDYT? cc @chenzl25
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 intend to move fast and present a demo version. Any optimization could be done in parallel as well. 😄
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.
Cool, let's move
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.
Sorry for replying late. Yes. And I agree to move it now. For the new API, let's track it in icelake as a new feature.