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 eq delete merge on read for iceberg #18448

Merged
merged 27 commits into from
Sep 13, 2024
Merged
Show file tree
Hide file tree
Changes from 8 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
179 changes: 67 additions & 112 deletions Cargo.lock

Large diffs are not rendered by default.

7 changes: 3 additions & 4 deletions Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -142,9 +142,9 @@ arrow-array-iceberg = { package = "arrow-array", version = "52" }
arrow-schema-iceberg = { package = "arrow-schema", version = "52" }
arrow-buffer-iceberg = { package = "arrow-buffer", version = "52" }
arrow-cast-iceberg = { package = "arrow-cast", version = "52" }
iceberg = "0.3.0"
iceberg-catalog-rest = "0.3.0"
iceberg-catalog-glue = "0.3.0"
iceberg = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "682b38037dafa78ebbc6e0479844e980c5ed68a4" }
iceberg-catalog-rest = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "682b38037dafa78ebbc6e0479844e980c5ed68a4" }
iceberg-catalog-glue = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "682b38037dafa78ebbc6e0479844e980c5ed68a4" }
Copy link
Collaborator

Choose a reason for hiding this comment

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

Just curious, do we have a plan to contribute back to upstream to expose delete files?

opendal = "0.47"
arrow-array = "50"
arrow-arith = "50"
Expand Down Expand Up @@ -342,7 +342,6 @@ opt-level = 2

[patch.crates-io]
# Patch third-party crates for deterministic simulation.
quanta = { git = "https://github.com/madsim-rs/quanta.git", rev = "948bdc3" }
getrandom = { git = "https://github.com/madsim-rs/getrandom.git", rev = "e79a7ae" }
# Don't patch `tokio-stream`, but only use the madsim version for **direct** dependencies.
# Imagine an unpatched dependency depends on the original `tokio` and the patched `tokio-stream`.
Expand Down
4 changes: 2 additions & 2 deletions risedev.yml
Original file line number Diff line number Diff line change
Expand Up @@ -115,8 +115,8 @@ profile:
- use: compactor
- use: prometheus
- use: grafana
- use: kafka
persist-data: true
# - use: kafka
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
# persist-data: true

standalone-full-peripherals:
steps:
Expand Down
155 changes: 132 additions & 23 deletions src/batch/src/executor/iceberg_scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.

use core::ops::BitOr;
use std::collections::HashMap;
use std::mem;

use futures_async_stream::try_stream;
Expand All @@ -20,6 +22,7 @@ use iceberg::scan::FileScanTask;
use iceberg::spec::TableMetadata;
use itertools::Itertools;
use risingwave_common::array::arrow::IcebergArrowConvert;
use risingwave_common::bitmap::Bitmap;
use risingwave_common::catalog::{Field, Schema};
use risingwave_common::types::DataType;
use risingwave_connector::sink::iceberg::IcebergConfig;
Expand All @@ -39,6 +42,7 @@ pub struct IcebergScanExecutor {
snapshot_id: Option<i64>,
table_meta: TableMetadata,
file_scan_tasks: Vec<FileScanTask>,
eq_delete_file_scan_tasks: Vec<FileScanTask>,
batch_size: usize,
schema: Schema,
identity: String,
Expand All @@ -64,6 +68,7 @@ impl IcebergScanExecutor {
snapshot_id: Option<i64>,
table_meta: TableMetadata,
file_scan_tasks: Vec<FileScanTask>,
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
eq_delete_file_scan_tasks: Vec<FileScanTask>,
batch_size: usize,
schema: Schema,
identity: String,
Expand All @@ -73,6 +78,7 @@ impl IcebergScanExecutor {
snapshot_id,
table_meta,
file_scan_tasks,
eq_delete_file_scan_tasks,
batch_size,
schema,
identity,
Expand All @@ -86,33 +92,131 @@ impl IcebergScanExecutor {
.load_table_v2_with_metadata(self.table_meta)
.await?;
let data_types = self.schema.data_types();
let chunk_schema_name_to_id = self
.schema
.names()
.iter()
.enumerate()
.map(|(k, v)| (v.clone(), k))
.collect::<HashMap<_, _>>();

let file_scan_tasks = mem::take(&mut self.file_scan_tasks);
let mut eq_delete_file_scan_tasks_map: HashMap<
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
String,
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
HashMap<Option<risingwave_common::types::ScalarImpl>, i64>,
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
> = HashMap::default();
let eq_delete_file_scan_tasks = mem::take(&mut self.eq_delete_file_scan_tasks);

for mut eq_delete_file_scan_task in eq_delete_file_scan_tasks {
eq_delete_file_scan_task.project_field_ids =
eq_delete_file_scan_task.equality_ids.clone();
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
let mut sequence_number = eq_delete_file_scan_task.sequence_number;
let reader = table
.reader_builder()
.with_batch_size(self.batch_size)
.build();
let delete_file_scan_stream = tokio_stream::once(Ok(eq_delete_file_scan_task));

let file_scan_stream = {
#[try_stream]
async move {
for file_scan_task in file_scan_tasks {
yield file_scan_task;
let mut delete_record_batch_stream = reader
.read(Box::pin(delete_file_scan_stream))
.map_err(BatchError::Iceberg)?;

while let Some(record_batch) = delete_record_batch_stream.next().await {
let record_batch = record_batch.map_err(BatchError::Iceberg)?;
let delete_column_names = record_batch
.schema()
.fields()
.iter()
.map(|field| field.name())
.cloned()
.collect_vec();

let chunk = IcebergArrowConvert.chunk_from_record_batch(&record_batch)?;
for (array, columns_name) in chunk.columns().iter().zip_eq(delete_column_names) {
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
let each_column_seq_num_map = eq_delete_file_scan_tasks_map
.entry(columns_name)
.or_default();
for datum in array.get_all_values() {
let entry = each_column_seq_num_map
.entry(datum)
.or_insert(sequence_number);
*entry = *entry.max(&mut sequence_number);
}
}
}
};

let reader = table
.reader_builder()
.with_batch_size(self.batch_size)
.build();

let record_batch_stream = reader
.read(Box::pin(file_scan_stream))
.map_err(BatchError::Iceberg)?;

#[for_await]
for record_batch in record_batch_stream {
let record_batch = record_batch.map_err(BatchError::Iceberg)?;
let chunk = IcebergArrowConvert.chunk_from_record_batch(&record_batch)?;
debug_assert_eq!(chunk.data_types(), data_types);
yield chunk;
}

let file_scan_tasks = mem::take(&mut self.file_scan_tasks);

for file_scan_task in file_scan_tasks {
let sequence_number = file_scan_task.sequence_number;
let reader = table
.reader_builder()
.with_batch_size(self.batch_size)
.build();
let file_scan_stream = tokio_stream::once(Ok(file_scan_task));

let mut record_batch_stream = reader
.read(Box::pin(file_scan_stream))
.map_err(BatchError::Iceberg)?;

while let Some(record_batch) = record_batch_stream.next().await {
let record_batch = record_batch.map_err(BatchError::Iceberg)?;
let column_names = record_batch
.schema()
.fields()
.iter()
.map(|field| field.name())
.cloned()
.collect_vec();
let chunk = IcebergArrowConvert.chunk_from_record_batch(&record_batch)?;
let visibilitys: Vec<_> = chunk
.columns()
.iter()
.zip_eq(column_names.clone())
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto

.filter_map(|(array, column_map)| {
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
if let Some(each_column_seq_num_map) =
eq_delete_file_scan_tasks_map.get(&column_map)
{
let visibility =
Bitmap::from_iter(array.get_all_values().iter().map(|datum| {
if let Some(s) = each_column_seq_num_map.get(datum)
&& s > &sequence_number
{
false
} else {
true
}
}));
Some(visibility)
} else {
None
}
})
.collect();
let (data, va) = chunk.into_parts_v2();
let visibility = if visibilitys.is_empty() {
va
} else {
visibilitys
.iter()
.skip(1)
.fold(visibilitys[0].clone(), |acc, bitmap| acc.bitor(bitmap))
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
};
let data = data
.iter()
.zip_eq(column_names)
.filter_map(|(array, columns)| {
chunk_schema_name_to_id
.get(&columns)
.map(|&id| (id, array.clone()))
})
.sorted_by_key(|a| a.0)
.map(|(k, v)| v)
.collect_vec();
let chunk = DataChunk::new(data, visibility);
debug_assert_eq!(chunk.data_types(), data_types);
yield chunk;
}
}
}
}
Expand Down Expand Up @@ -171,6 +275,11 @@ impl BoxedExecutorBuilder for IcebergScanExecutorBuilder {
Some(split.snapshot_id),
split.table_meta.deserialize(),
split.files.into_iter().map(|x| x.deserialize()).collect(),
split
.eq_delete_files
.into_iter()
.map(|x| x.deserialize())
.collect(),
source.context.get_config().developer.chunk_size,
schema,
source.plan_node().get_identity().clone(),
Expand Down
4 changes: 4 additions & 0 deletions src/common/src/array/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -603,6 +603,10 @@ impl ArrayImpl {
})
}

pub fn get_all_values(&self) -> Vec<Datum> {
(0..self.len()).map(|i| self.datum_at(i)).collect()
}

/// # Safety
xxhZs marked this conversation as resolved.
Show resolved Hide resolved
///
/// This function is unsafe because it does not check the validity of `idx`. It is caller's
Expand Down
2 changes: 1 addition & 1 deletion src/connector/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ jni = { version = "0.21.1", features = ["invocation"] }
jsonbb = { workspace = true }
jsonwebtoken = "9.2.0"
maplit = "1.0.2"
moka = { version = "0.12", features = ["future"] }
moka = { version = "0.12.8", features = ["future"] }
mongodb = { version = "2.8.2", features = ["tokio-runtime"] }
mysql_async = { version = "0.34", default-features = false, features = [
"default",
Expand Down
4 changes: 2 additions & 2 deletions src/connector/src/sink/iceberg/jni_catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -288,7 +288,7 @@ impl CatalogV2 for JniCatalog {
"Failed to crete iceberg table.",
)
.with_source(e)
})
})?
}

/// Load table from the catalog.
Expand Down Expand Up @@ -338,7 +338,7 @@ impl CatalogV2 for JniCatalog {
"Failed to load iceberg table.",
)
.with_source(e)
})
})?
}

/// Drop a table from the catalog.
Expand Down
2 changes: 1 addition & 1 deletion src/connector/src/sink/iceberg/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -672,7 +672,7 @@ impl IcebergConfig {
.file_io(storage_catalog.file_io().clone())
// Only support readonly table for storage catalog now.
.readonly(true)
.build())
.build()?)
}
_ => self.load_table_v2().await,
}
Expand Down
8 changes: 4 additions & 4 deletions src/connector/src/sink/iceberg/storage_catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -249,11 +249,11 @@ impl Catalog for StorageCatalog {
let version_hint_output = self.file_io.new_output(&version_hint_path)?;
version_hint_output.write("1".into()).await?;

Ok(Table::builder()
Table::builder()
.metadata(table_metadata)
.identifier(table_ident)
.file_io(self.file_io.clone())
.build())
.build()
}

/// Load table from the catalog.
Expand Down Expand Up @@ -283,13 +283,13 @@ impl Catalog for StorageCatalog {
let metadata_file_content = metadata_file.read().await?;
let table_metadata = serde_json::from_slice::<TableMetadata>(&metadata_file_content)?;

Ok(Table::builder()
Table::builder()
.metadata(table_metadata)
.identifier(table.clone())
.file_io(self.file_io.clone())
// Only support readonly table for storage catalog now.
.readonly(true)
.build())
.build()
}

/// Drop a table from the catalog.
Expand Down
Loading
Loading