forked from apache/iceberg-rust
-
Notifications
You must be signed in to change notification settings - Fork 0
feat(core): Add incremental scan for appends and positional deletes #3
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
Merged
Merged
Changes from all commits
Commits
Show all changes
76 commits
Select commit
Hold shift + click to select a range
fb5c737
WIP, initial draft of incremental scan
gbrgr 0fec28d
Merge branch 'main' into gb/incremental-bootstrap
gbrgr f255405
.
gbrgr be932c8
.
gbrgr a378abe
cargo fmt
gbrgr 317b5c6
Implement unzipped stream
gbrgr 06694a8
Remove printlns
gbrgr 4514695
Add API method for unzipped stream
gbrgr f56e068
.
gbrgr 4125328
Remove comment
gbrgr 9fa6360
Rename var
gbrgr 14dd4e9
Add import
gbrgr 837947a
Measure time
gbrgr 5a7a223
Fix typo
gbrgr 065c754
Undo some changes
gbrgr f9ea05c
Change type name
gbrgr 06cba66
Add comment header
gbrgr d3d57e3
Merge branch 'main' into gb/incremental-bootstrap
gbrgr 2ddde41
Fail when encountering equality deletes
gbrgr facbeeb
Add comments
gbrgr ae76630
Add some preliminary tests
gbrgr 2780634
Format
gbrgr aeeab92
Merge branch 'main' into gb/incremental-bootstrap
gbrgr 536bbc4
Remove playground
gbrgr 1f35b46
Merge branch 'gb/incremental-bootstrap' of github.com:RelationalAI/ic…
gbrgr 5ac1887
Add more tests
gbrgr 1f72025
Clippy
gbrgr 6d72c8c
.
gbrgr a024e60
.
gbrgr 773a9d0
Adapt tests
gbrgr 416a56d
.
gbrgr e06c118
Add test
gbrgr dc56ff7
Add tests
gbrgr 3b2b6a0
Add tests
gbrgr af28705
Format
gbrgr feb1cfe
Add test
gbrgr dd1eec8
Format
gbrgr ba5d3b1
.
gbrgr d8e6bc9
Rm newline
gbrgr 32cf060
Rename trait function
gbrgr 3b57c88
Reuse schema
gbrgr 07a4394
.
gbrgr 87587d6
remove clone
gbrgr 1e66b4b
Add test for adding file_path column
gbrgr b77f37f
Make `from_snapshot` mandatory
gbrgr 2204b5c
Error out if incremental scan encounters neither Append nor Delete
gbrgr a281d7f
.
gbrgr 7240249
Add materialized variant of add_file_path_column
gbrgr 170e0cf
.
gbrgr c02af9e
Allow dead code
gbrgr 299e274
Some PR comments
gbrgr 2c77259
.
gbrgr 70683de
More PR comments
gbrgr e4ad209
.
gbrgr 279df44
Add comments
gbrgr 630d623
Avoid cloning
gbrgr 337c73e
Add reference to PR
gbrgr 18ff3b3
Merge branch 'main' into gb/incremental-bootstrap
gbrgr e31f76e
Some PR comments
gbrgr 02f9a81
.
gbrgr ec24817
format
gbrgr efc57e8
Allow overwrite operation for now
gbrgr f78fcca
Fix file_path column
gbrgr 97efb4c
Add overwrite test
gbrgr 055b9b0
Merge branch 'main' into gb/incremental-bootstrap
gbrgr 3aa40cb
Unwrap delete vector
gbrgr 584a684
.
gbrgr e666dfd
Merge branch 'gb/incremental-bootstrap' of github.com:RelationalAI/ic…
gbrgr ff8a38c
Add assertion
gbrgr f31bb64
Avoid cloning the mutex guard
gbrgr c6f5ab6
Abort when encountering a deleted delete file
gbrgr c0989b0
Adjust comment
gbrgr b95414c
Update crates/iceberg/src/arrow/reader.rs
gbrgr 87da883
Add check
gbrgr 4982ee2
Merge branch 'gb/incremental-bootstrap' of github.com:RelationalAI/ic…
gbrgr 6c60b90
Update crates/iceberg/src/scan/incremental/mod.rs
vustef 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
This file contains hidden or 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 hidden or 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,271 @@ | ||
| // Licensed to the Apache Software Foundation (ASF) under one | ||
| // or more contributor license agreements. See the NOTICE file | ||
| // distributed with this work for additional information | ||
| // regarding copyright ownership. The ASF licenses this file | ||
| // to you 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::pin::Pin; | ||
| use std::sync::Arc; | ||
|
|
||
| use arrow_array::{RecordBatch, UInt64Array}; | ||
| use arrow_schema::{DataType, Field, Schema as ArrowSchema}; | ||
| use futures::channel::mpsc::channel; | ||
| use futures::stream::select; | ||
| use futures::{SinkExt, Stream, StreamExt, TryStreamExt}; | ||
|
|
||
| use crate::arrow::record_batch_transformer::RecordBatchTransformer; | ||
| use crate::arrow::{ | ||
| ArrowReader, RESERVED_COL_NAME_FILE_PATH, RESERVED_FIELD_ID_FILE_PATH, StreamsInto, | ||
| }; | ||
| use crate::delete_vector::DeleteVector; | ||
| use crate::io::FileIO; | ||
| use crate::runtime::spawn; | ||
| use crate::scan::ArrowRecordBatchStream; | ||
| use crate::scan::incremental::{ | ||
| AppendedFileScanTask, IncrementalFileScanTask, IncrementalFileScanTaskStream, | ||
| }; | ||
| use crate::{Error, ErrorKind, Result}; | ||
|
|
||
| /// The type of incremental batch: appended data or deleted records. | ||
| #[derive(Debug, Clone, Copy, PartialEq, Eq)] | ||
| pub enum IncrementalBatchType { | ||
| /// Appended records. | ||
| Append, | ||
| /// Deleted records. | ||
| Delete, | ||
| } | ||
|
|
||
| /// The stream of incremental Arrow `RecordBatch`es with batch type. | ||
| pub type CombinedIncrementalBatchRecordStream = | ||
| Pin<Box<dyn Stream<Item = Result<(IncrementalBatchType, RecordBatch)>> + Send + 'static>>; | ||
|
|
||
| /// Stream type for obtaining a separate stream of appended and deleted record batches. | ||
| pub type UnzippedIncrementalBatchRecordStream = (ArrowRecordBatchStream, ArrowRecordBatchStream); | ||
|
|
||
| impl StreamsInto<ArrowReader, CombinedIncrementalBatchRecordStream> | ||
| for IncrementalFileScanTaskStream | ||
| { | ||
| /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns a | ||
| /// stream of Arrow `RecordBatch`es containing the data from the files. | ||
| fn stream(self, reader: ArrowReader) -> Result<CombinedIncrementalBatchRecordStream> { | ||
| let (appends, deletes) = | ||
| StreamsInto::<ArrowReader, UnzippedIncrementalBatchRecordStream>::stream(self, reader)?; | ||
|
|
||
| let left = appends.map(|res| res.map(|batch| (IncrementalBatchType::Append, batch))); | ||
| let right = deletes.map(|res| res.map(|batch| (IncrementalBatchType::Delete, batch))); | ||
|
|
||
| Ok(Box::pin(select(left, right)) as CombinedIncrementalBatchRecordStream) | ||
| } | ||
| } | ||
|
|
||
| impl StreamsInto<ArrowReader, UnzippedIncrementalBatchRecordStream> | ||
| for IncrementalFileScanTaskStream | ||
| { | ||
| /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns two | ||
| /// separate streams of Arrow `RecordBatch`es containing appended data and deleted records. | ||
| fn stream(self, reader: ArrowReader) -> Result<UnzippedIncrementalBatchRecordStream> { | ||
| let (appends_tx, appends_rx) = channel(reader.concurrency_limit_data_files); | ||
| let (deletes_tx, deletes_rx) = channel(reader.concurrency_limit_data_files); | ||
|
|
||
| let batch_size = reader.batch_size; | ||
| let concurrency_limit_data_files = reader.concurrency_limit_data_files; | ||
|
|
||
| spawn(async move { | ||
| let _ = self | ||
| .try_for_each_concurrent(concurrency_limit_data_files, |task| { | ||
| let file_io = reader.file_io.clone(); | ||
| let mut appends_tx = appends_tx.clone(); | ||
| let mut deletes_tx = deletes_tx.clone(); | ||
| async move { | ||
| match task { | ||
| IncrementalFileScanTask::Append(append_task) => { | ||
| spawn(async move { | ||
| let record_batch_stream = process_incremental_append_task( | ||
| append_task, | ||
| batch_size, | ||
| file_io, | ||
| ) | ||
| .await; | ||
|
|
||
| match record_batch_stream { | ||
| Ok(mut stream) => { | ||
| while let Some(batch) = stream.next().await { | ||
| let result = appends_tx | ||
| .send(batch.map_err(|e| { | ||
| Error::new( | ||
| ErrorKind::Unexpected, | ||
| "failed to read appended record batch", | ||
| ) | ||
| .with_source(e) | ||
| })) | ||
| .await; | ||
|
|
||
| if result.is_err() { | ||
| break; | ||
| } | ||
| } | ||
| } | ||
| Err(e) => { | ||
| let _ = appends_tx.send(Err(e)).await; | ||
| } | ||
| } | ||
| }); | ||
| } | ||
| IncrementalFileScanTask::Delete(file_path, delete_vector) => { | ||
| spawn(async move { | ||
| let record_batch_stream = process_incremental_delete_task( | ||
| file_path, | ||
| delete_vector, | ||
| batch_size, | ||
| ); | ||
|
|
||
| match record_batch_stream { | ||
| Ok(mut stream) => { | ||
| while let Some(batch) = stream.next().await { | ||
| let result = deletes_tx | ||
| .send(batch.map_err(|e| { | ||
| Error::new( | ||
| ErrorKind::Unexpected, | ||
| "failed to read deleted record batch", | ||
| ) | ||
| .with_source(e) | ||
| })) | ||
| .await; | ||
|
|
||
| if result.is_err() { | ||
| break; | ||
| } | ||
| } | ||
| } | ||
| Err(e) => { | ||
| let _ = deletes_tx.send(Err(e)).await; | ||
| } | ||
| } | ||
| }); | ||
| } | ||
| }; | ||
|
|
||
| Ok(()) | ||
| } | ||
| }) | ||
| .await; | ||
| }); | ||
|
|
||
| Ok(( | ||
| Box::pin(appends_rx) as ArrowRecordBatchStream, | ||
| Box::pin(deletes_rx) as ArrowRecordBatchStream, | ||
| )) | ||
| } | ||
| } | ||
|
|
||
| async fn process_incremental_append_task( | ||
| task: AppendedFileScanTask, | ||
| batch_size: Option<usize>, | ||
| file_io: FileIO, | ||
| ) -> Result<ArrowRecordBatchStream> { | ||
| let mut record_batch_stream_builder = ArrowReader::create_parquet_record_batch_stream_builder( | ||
| &task.data_file_path, | ||
| file_io, | ||
| true, | ||
| ) | ||
| .await?; | ||
|
|
||
| // Create a projection mask for the batch stream to select which columns in the | ||
| // Parquet file that we want in the response | ||
| let projection_mask = ArrowReader::get_arrow_projection_mask( | ||
| &task.project_field_ids, | ||
| &task.schema_ref(), | ||
| record_batch_stream_builder.parquet_schema(), | ||
| record_batch_stream_builder.schema(), | ||
| )?; | ||
| record_batch_stream_builder = record_batch_stream_builder.with_projection(projection_mask); | ||
|
|
||
| // RecordBatchTransformer performs any transformations required on the RecordBatches | ||
| // that come back from the file, such as type promotion, default column insertion | ||
| // and column re-ordering | ||
| let mut record_batch_transformer = | ||
| RecordBatchTransformer::build(task.schema_ref(), &task.project_field_ids); | ||
|
|
||
| if let Some(batch_size) = batch_size { | ||
| record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); | ||
| } | ||
|
|
||
| // Apply positional deletes as row selections. | ||
| let row_selection = if let Some(positional_delete_indexes) = task.positional_deletes { | ||
| Some(ArrowReader::build_deletes_row_selection( | ||
| record_batch_stream_builder.metadata().row_groups(), | ||
| &None, | ||
| &positional_delete_indexes.lock().unwrap(), | ||
| )?) | ||
| } else { | ||
| None | ||
| }; | ||
|
|
||
| if let Some(row_selection) = row_selection { | ||
| record_batch_stream_builder = record_batch_stream_builder.with_row_selection(row_selection); | ||
| } | ||
|
|
||
| // Build the batch stream and send all the RecordBatches that it generates | ||
| // to the requester. | ||
| let record_batch_stream = record_batch_stream_builder | ||
| .build()? | ||
| .map(move |batch| match batch { | ||
| Ok(batch) => record_batch_transformer.process_record_batch(batch), | ||
| Err(err) => Err(err.into()), | ||
| }); | ||
|
|
||
| Ok(Box::pin(record_batch_stream) as ArrowRecordBatchStream) | ||
| } | ||
|
|
||
| fn process_incremental_delete_task( | ||
| file_path: String, | ||
| delete_vector: DeleteVector, | ||
| batch_size: Option<usize>, | ||
| ) -> Result<ArrowRecordBatchStream> { | ||
| let schema = Arc::new(ArrowSchema::new(vec![Field::new( | ||
| "pos", | ||
gbrgr marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| DataType::UInt64, | ||
| false, | ||
| )])); | ||
|
|
||
| let batch_size = batch_size.unwrap_or(1024); | ||
|
|
||
| let treemap = delete_vector.inner; | ||
|
|
||
| let stream = futures::stream::iter(treemap) | ||
| .chunks(batch_size) | ||
| .map(move |chunk| { | ||
| let array = UInt64Array::from_iter(chunk); | ||
| RecordBatch::try_new( | ||
| Arc::clone(&schema), // Cheap Arc clone instead of full schema creation | ||
| vec![Arc::new(array)], | ||
| ) | ||
| .map_err(|_| { | ||
| Error::new( | ||
| ErrorKind::Unexpected, | ||
| "Failed to create RecordBatch for DeleteVector", | ||
| ) | ||
| }) | ||
| .and_then(|batch| { | ||
| ArrowReader::add_file_path_column( | ||
| batch, | ||
| &file_path, | ||
| RESERVED_COL_NAME_FILE_PATH, | ||
| RESERVED_FIELD_ID_FILE_PATH, | ||
| ) | ||
| }) | ||
| }); | ||
|
|
||
| Ok(Box::pin(stream) as ArrowRecordBatchStream) | ||
| } | ||
This file contains hidden or 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.
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.
Uh oh!
There was an error while loading. Please reload this page.