-
Notifications
You must be signed in to change notification settings - Fork 590
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): Optimization iceberg source count(*) query #18067
Changes from all commits
4f41f92
6eb7471
a2ec259
fadc084
1c526be
497352e
ae95e1e
0025df1
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,64 @@ | ||
// 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::mem; | ||
use std::sync::Arc; | ||
|
||
use futures_async_stream::try_stream; | ||
use futures_util::stream::StreamExt; | ||
use risingwave_common::array::I64Array; | ||
use risingwave_common::catalog::Schema; | ||
|
||
use crate::error::BatchError; | ||
use crate::executor::{DataChunk, Executor}; | ||
|
||
pub struct IcebergCountStarExecutor { | ||
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. It seems we don't even need a special iceberg count start executor, |
||
schema: Schema, | ||
identity: String, | ||
record_counts: Vec<u64>, | ||
} | ||
|
||
impl Executor for IcebergCountStarExecutor { | ||
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 IcebergCountStarExecutor { | ||
pub fn new(schema: Schema, identity: String, record_counts: Vec<u64>) -> Self { | ||
Self { | ||
schema, | ||
identity, | ||
record_counts, | ||
} | ||
} | ||
|
||
#[try_stream(ok = DataChunk, error = BatchError)] | ||
async fn do_execute(mut self: Box<Self>) { | ||
let record_count = mem::take(&mut self.record_counts).into_iter().sum::<u64>() as i64; | ||
let chunk = DataChunk::new( | ||
vec![Arc::new(I64Array::from_iter([record_count]).into())], | ||
1, | ||
); | ||
yield chunk; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,14 +20,15 @@ use anyhow::anyhow; | |
use async_trait::async_trait; | ||
use futures_async_stream::for_await; | ||
use iceberg::scan::FileScanTask; | ||
use iceberg::spec::TableMetadata; | ||
use iceberg::spec::{ManifestList, TableMetadata}; | ||
use iceberg::table::Table; | ||
use itertools::Itertools; | ||
pub use parquet_file_reader::*; | ||
use risingwave_common::bail; | ||
use risingwave_common::catalog::Schema; | ||
use risingwave_common::types::JsonbVal; | ||
use serde::{Deserialize, Serialize}; | ||
use tokio_stream::StreamExt; | ||
|
||
use crate::error::{ConnectorError, ConnectorResult}; | ||
use crate::parser::ParserConfig; | ||
|
@@ -145,6 +146,7 @@ pub struct IcebergSplit { | |
pub snapshot_id: i64, | ||
pub table_meta: TableMetadataJsonStr, | ||
pub files: Vec<IcebergFileScanTaskJsonStr>, | ||
pub record_counts: Vec<u64>, | ||
pub eq_delete_files: Vec<IcebergFileScanTaskJsonStr>, | ||
} | ||
|
||
|
@@ -198,63 +200,90 @@ pub enum IcebergTimeTravelInfo { | |
} | ||
|
||
impl IcebergSplitEnumerator { | ||
pub async fn list_splits_batch( | ||
pub async fn list_splits_batch_count_star( | ||
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. If it is a count star query, we don't need to return multiple splits, i.e. we don't need to run it in a distributed way. |
||
&self, | ||
schema: Schema, | ||
time_traval_info: Option<IcebergTimeTravelInfo>, | ||
batch_parallelism: usize, | ||
) -> ConnectorResult<Vec<IcebergSplit>> { | ||
if batch_parallelism == 0 { | ||
bail!("Batch parallelism is 0. Cannot split the iceberg files."); | ||
} | ||
let table = self.config.load_table_v2().await?; | ||
|
||
let current_snapshot = table.metadata().current_snapshot(); | ||
if current_snapshot.is_none() { | ||
let table_meta = TableMetadataJsonStr::serialize(table.metadata()); | ||
let Some(snapshot_id) = self.get_snapshot_id(&table, time_traval_info)? else{ | ||
// If there is no snapshot, we will return a mock `IcebergSplit` with empty files. | ||
return Ok(vec![IcebergSplit { | ||
split_id: 0, | ||
snapshot_id: 0, // unused | ||
table_meta: TableMetadataJsonStr::serialize(table.metadata()), | ||
return Ok(vec![IcebergSplit {split_id:0,snapshot_id:0,table_meta,files:vec![],eq_delete_files:vec![], record_counts: vec![] }]); | ||
}; | ||
let mut record_counts = vec![]; | ||
let manifest_list: ManifestList = table | ||
.metadata() | ||
.snapshot_by_id(snapshot_id) | ||
.unwrap() | ||
.load_manifest_list(table.file_io(), table.metadata()) | ||
.await | ||
.map_err(|e| anyhow!(e))?; | ||
|
||
for entry in manifest_list.entries() { | ||
let manifest = entry | ||
.load_manifest(table.file_io()) | ||
.await | ||
.map_err(|e| anyhow!(e))?; | ||
let mut manifest_entries_stream = | ||
futures::stream::iter(manifest.entries().iter().filter(|e| e.is_alive())); | ||
|
||
while let Some(manifest_entry) = manifest_entries_stream.next().await { | ||
let file = manifest_entry.data_file(); | ||
record_counts.push(file.record_count()); | ||
} | ||
} | ||
let split_num = batch_parallelism; | ||
// evenly split the files into splits based on the parallelism. | ||
let split_size = record_counts.len() / split_num; | ||
let remaining = record_counts.len() % split_num; | ||
let mut splits = vec![]; | ||
for i in 0..split_num { | ||
let start = i * split_size; | ||
let end = (i + 1) * split_size; | ||
let split = IcebergSplit { | ||
split_id: i as i64, | ||
snapshot_id, | ||
table_meta: table_meta.clone(), | ||
files: vec![], | ||
record_counts: record_counts[start..end].to_vec(), | ||
eq_delete_files: vec![], | ||
}]); | ||
}; | ||
splits.push(split); | ||
} | ||
for i in 0..remaining { | ||
splits[i] | ||
.record_counts | ||
.push(record_counts[split_num * split_size + i]); | ||
} | ||
|
||
let snapshot_id = match time_traval_info { | ||
Some(IcebergTimeTravelInfo::Version(version)) => { | ||
let Some(snapshot) = table.metadata().snapshot_by_id(version) else { | ||
bail!("Cannot find the snapshot id in the iceberg table."); | ||
}; | ||
snapshot.snapshot_id() | ||
} | ||
Some(IcebergTimeTravelInfo::TimestampMs(timestamp)) => { | ||
let snapshot = table | ||
.metadata() | ||
.snapshots() | ||
.map(|snapshot| snapshot.timestamp().map(|ts| ts.timestamp_millis())) | ||
.collect::<Result<Vec<_>, _>>()? | ||
.into_iter() | ||
.filter(|&snapshot_millis| snapshot_millis <= timestamp) | ||
.max_by_key(|&snapshot_millis| snapshot_millis); | ||
match snapshot { | ||
Some(snapshot) => snapshot, | ||
None => { | ||
// convert unix time to human readable time | ||
let time = chrono::DateTime::from_timestamp_millis(timestamp); | ||
if time.is_some() { | ||
bail!("Cannot find a snapshot older than {}", time.unwrap()); | ||
} else { | ||
bail!("Cannot find a snapshot"); | ||
} | ||
} | ||
} | ||
} | ||
None => { | ||
assert!(current_snapshot.is_some()); | ||
current_snapshot.unwrap().snapshot_id() | ||
} | ||
Ok(splits | ||
.into_iter() | ||
.filter(|split| !split.record_counts.is_empty()) | ||
.collect_vec()) | ||
} | ||
|
||
pub async fn list_splits_batch_scan( | ||
&self, | ||
schema: Schema, | ||
time_traval_info: Option<IcebergTimeTravelInfo>, | ||
batch_parallelism: usize, | ||
) -> ConnectorResult<Vec<IcebergSplit>> { | ||
if batch_parallelism == 0 { | ||
bail!("Batch parallelism is 0. Cannot split the iceberg files."); | ||
} | ||
let table = self.config.load_table_v2().await?; | ||
|
||
let table_meta = TableMetadataJsonStr::serialize(table.metadata()); | ||
|
||
let Some(snapshot_id) = self.get_snapshot_id(&table, time_traval_info)? else{ | ||
// If there is no snapshot, we will return a mock `IcebergSplit` with empty files. | ||
return Ok(vec![IcebergSplit {split_id:0,snapshot_id:0,table_meta,files:vec![],eq_delete_files:vec![], record_counts: vec![] }]); | ||
}; | ||
|
||
let require_names = Self::get_require_field_names(&table, snapshot_id, schema).await?; | ||
|
||
let mut data_files = vec![]; | ||
|
@@ -286,8 +315,6 @@ impl IcebergSplitEnumerator { | |
} | ||
} | ||
|
||
let table_meta = TableMetadataJsonStr::serialize(table.metadata()); | ||
|
||
let split_num = batch_parallelism; | ||
// evenly split the files into splits based on the parallelism. | ||
let split_size = data_files.len() / split_num; | ||
|
@@ -300,6 +327,7 @@ impl IcebergSplitEnumerator { | |
split_id: i as i64, | ||
snapshot_id, | ||
table_meta: table_meta.clone(), | ||
record_counts: vec![], | ||
files: data_files[start..end].to_vec(), | ||
eq_delete_files: eq_delete_files.clone(), | ||
}; | ||
|
@@ -356,6 +384,48 @@ impl IcebergSplitEnumerator { | |
} | ||
Ok(require_field_names) | ||
} | ||
|
||
fn get_snapshot_id(&self, table: &Table, time_traval_info: Option<IcebergTimeTravelInfo>) -> ConnectorResult<Option<i64>> { | ||
let current_snapshot = table.metadata().current_snapshot(); | ||
if current_snapshot.is_none() { | ||
// If there is no snapshot, we will return a mock `IcebergSplit` with empty files. | ||
return Ok(None); | ||
} | ||
match time_traval_info { | ||
Some(IcebergTimeTravelInfo::Version(version)) => { | ||
let Some(snapshot) = table.metadata().snapshot_by_id(version) else { | ||
bail!("Cannot find the snapshot id in the iceberg table."); | ||
}; | ||
Ok(Some(snapshot.snapshot_id())) | ||
} | ||
Some(IcebergTimeTravelInfo::TimestampMs(timestamp)) => { | ||
let snapshot = table | ||
.metadata() | ||
.snapshots() | ||
.map(|snapshot| snapshot.timestamp().map(|ts| ts.timestamp_millis())) | ||
.collect::<Result<Vec<_>, _>>()? | ||
.into_iter() | ||
.filter(|&snapshot_millis| snapshot_millis <= timestamp) | ||
.max_by_key(|&snapshot_millis| snapshot_millis); | ||
match snapshot { | ||
Some(snapshot) => Ok(Some(snapshot)), | ||
None => { | ||
// convert unix time to human readable time | ||
let time = chrono::DateTime::from_timestamp_millis(timestamp); | ||
if time.is_some() { | ||
bail!("Cannot find a snapshot older than {}", time.unwrap()); | ||
} else { | ||
bail!("Cannot find a snapshot"); | ||
} | ||
} | ||
} | ||
} | ||
None => { | ||
assert!(current_snapshot.is_some()); | ||
Ok(Some(current_snapshot.unwrap().snapshot_id())) | ||
} | ||
} | ||
} | ||
} | ||
|
||
#[derive(Debug)] | ||
|
@@ -380,3 +450,4 @@ impl SplitReader for IcebergFileReader { | |
unimplemented!() | ||
} | ||
} | ||
|
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 this new field is too special to be a part of the
SourceNode
proto because it only meaningful for iceberg. Considering we will try to support predicate pushdown for iceberg scan, I think we can separate iceberg scan from source node just like file scan.risingwave/src/frontend/src/optimizer/plan_node/batch_file_scan.rs
Line 80 in 2e1d910