-
Couldn't load subscription status.
- Fork 118
feat: Add write_parquet_file to ParquetHandler
#1392
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
Open
Fokko
wants to merge
13
commits into
delta-io:main
Choose a base branch
from
Fokko:fd-write-parquet
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
13 commits
Select commit
Hold shift + click to select a range
ce1bb10
feat: Add `write_parquet_file` to `ParquetHandler`
Fokko 4c86db5
Change data to an Iterator of `FilteredEngineData`
Fokko fdca7ae
Thanks Stephen
Fokko 9dde642
Merge branch 'main' into fd-write-parquet
Fokko 4279a4b
Merge branch 'main' into fd-write-parquet
Fokko 10fee3c
Merge branch 'main' into fd-write-parquet
Fokko 6785266
Fix tests and remove empty check
Fokko b55a603
Fix tests and remove empty check
Fokko 1cad8e2
Filter the batch
Fokko 72b7e49
Merge branch 'fd-write-parquet' of github.com:Fokko/delta-kernel-rs i…
Fokko 425bd1f
Thanks Micah!
Fokko f6004fa
Merge branch 'main' into fd-write-parquet
Fokko 904c01e
Thanks Ruzel
Fokko 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 |
|---|---|---|
|
|
@@ -22,15 +22,15 @@ use super::UrlExt; | |
| use crate::engine::arrow_conversion::TryIntoArrow as _; | ||
| use crate::engine::arrow_data::ArrowEngineData; | ||
| use crate::engine::arrow_utils::{ | ||
| fixup_parquet_read, generate_mask, get_requested_indices, ordering_needs_row_indexes, | ||
| RowIndexBuilder, | ||
| filter_to_record_batch, fixup_parquet_read, generate_mask, get_requested_indices, | ||
| ordering_needs_row_indexes, RowIndexBuilder, | ||
| }; | ||
| use crate::engine::default::executor::TaskExecutor; | ||
| use crate::engine::parquet_row_group_skipping::ParquetRowGroupSkipping; | ||
| use crate::schema::SchemaRef; | ||
| use crate::{ | ||
| DeltaResult, EngineData, Error, FileDataReadResultIterator, FileMeta, ParquetHandler, | ||
| PredicateRef, | ||
| DeltaResult, EngineData, Error, FileDataReadResultIterator, FileMeta, FilteredEngineData, | ||
| ParquetHandler, PredicateRef, | ||
| }; | ||
|
|
||
| #[derive(Debug)] | ||
|
|
@@ -242,6 +242,45 @@ impl<E: TaskExecutor> ParquetHandler for DefaultParquetHandler<E> { | |
| self.readahead, | ||
| ) | ||
| } | ||
|
|
||
| fn write_parquet_file( | ||
| &self, | ||
| location: url::Url, | ||
| data: FilteredEngineData, | ||
| ) -> DeltaResult<FileMeta> { | ||
| // Convert FilteredEngineData to RecordBatch, applying selection filter | ||
| let batch = filter_to_record_batch(data)?; | ||
|
|
||
| // We buffer it in the application first, and then push everything to the object-store. | ||
| // The storage API does not seem to allow for streaming to the store, which is okay | ||
| // since often the object stores allocate their own buffers. | ||
| let mut buffer = Vec::new(); | ||
|
|
||
| // Scope to ensure writer is dropped before we use buffer | ||
| { | ||
| let mut writer = ArrowWriter::try_new(&mut buffer, batch.schema(), None)?; | ||
| writer.write(&batch)?; | ||
| writer.close()?; // writer must be closed to write footer | ||
| } | ||
|
|
||
| let store = self.store.clone(); | ||
| let path = Path::from_url_path(location.path())?; | ||
|
|
||
| let size: u64 = buffer | ||
| .len() | ||
| .try_into() | ||
| .map_err(|_| Error::generic("unable to convert usize to u64"))?; | ||
|
|
||
| // Block on the async put operation | ||
| self.task_executor | ||
| .block_on(async move { store.put(&path, buffer.into()).await })?; | ||
|
|
||
| Ok(FileMeta { | ||
| location, | ||
| last_modified: 0, | ||
| size, | ||
| }) | ||
| } | ||
| } | ||
|
|
||
| /// Implements [`FileOpener`] for a parquet file | ||
|
|
@@ -427,7 +466,7 @@ mod tests { | |
| use std::path::PathBuf; | ||
| use std::slice; | ||
|
|
||
| use crate::arrow::array::{Array, RecordBatch}; | ||
| use crate::arrow::array::{Array, BooleanArray, RecordBatch}; | ||
|
|
||
| use crate::engine::arrow_conversion::TryIntoKernel as _; | ||
| use crate::engine::arrow_data::ArrowEngineData; | ||
|
|
@@ -638,4 +677,159 @@ mod tests { | |
| "Generic delta kernel error: Path must end with a trailing slash: memory:///data", | ||
| ); | ||
| } | ||
|
|
||
| #[tokio::test] | ||
| async fn test_parquet_handler_trait_write() { | ||
| let store = Arc::new(InMemory::new()); | ||
| let parquet_handler: Arc<dyn ParquetHandler> = Arc::new(DefaultParquetHandler::new( | ||
| store.clone(), | ||
| Arc::new(TokioBackgroundExecutor::new()), | ||
| )); | ||
|
|
||
| let engine_data: Box<dyn EngineData> = Box::new(ArrowEngineData::new( | ||
| RecordBatch::try_from_iter(vec![ | ||
| ( | ||
| "x", | ||
| Arc::new(Int64Array::from(vec![10, 20, 30])) as Arc<dyn Array>, | ||
| ), | ||
| ( | ||
| "y", | ||
| Arc::new(Int64Array::from(vec![100, 200, 300])) as Arc<dyn Array>, | ||
| ), | ||
| ]) | ||
| .unwrap(), | ||
| )); | ||
|
|
||
| // Wrap in FilteredEngineData with all rows selected | ||
| let filtered_data = crate::FilteredEngineData::with_all_rows_selected(engine_data); | ||
|
|
||
| // Test writing through the trait method | ||
| let file_url = Url::parse("memory:///test/data.parquet").unwrap(); | ||
| parquet_handler | ||
| .write_parquet_file(file_url.clone(), filtered_data) | ||
| .unwrap(); | ||
|
|
||
| // Verify we can read the file back | ||
| let path = Path::from_url_path(file_url.path()).unwrap(); | ||
| let reader = ParquetObjectReader::new(store.clone(), path); | ||
| let physical_schema = ParquetRecordBatchStreamBuilder::new(reader) | ||
| .await | ||
| .unwrap() | ||
| .schema() | ||
| .clone(); | ||
|
|
||
| let file_meta = FileMeta { | ||
| location: file_url, | ||
| last_modified: 0, | ||
| size: 0, | ||
| }; | ||
|
|
||
| let data: Vec<RecordBatch> = parquet_handler | ||
| .read_parquet_files( | ||
| slice::from_ref(&file_meta), | ||
| Arc::new(physical_schema.try_into_kernel().unwrap()), | ||
| None, | ||
| ) | ||
| .unwrap() | ||
| .map(into_record_batch) | ||
| .try_collect() | ||
| .unwrap(); | ||
|
|
||
| assert_eq!(data.len(), 1); | ||
|
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. I think we also need verify that field-id's are populated and we can project based on field IDs for column indirection? Will this be a follow-up? |
||
| assert_eq!(data[0].num_rows(), 3); | ||
| assert_eq!(data[0].num_columns(), 2); | ||
| } | ||
|
|
||
| #[tokio::test] | ||
| async fn test_parquet_handler_trait_write_and_read_roundtrip() { | ||
| let store = Arc::new(InMemory::new()); | ||
| let parquet_handler: Arc<dyn ParquetHandler> = Arc::new(DefaultParquetHandler::new( | ||
| store.clone(), | ||
| Arc::new(TokioBackgroundExecutor::new()), | ||
| )); | ||
|
|
||
| // Create test data with multiple types | ||
| let engine_data: Box<dyn EngineData> = Box::new(ArrowEngineData::new( | ||
| RecordBatch::try_from_iter(vec![ | ||
| ( | ||
| "int_col", | ||
| Arc::new(Int64Array::from(vec![1, 2, 3, 4, 5])) as Arc<dyn Array>, | ||
| ), | ||
| ( | ||
| "string_col", | ||
| Arc::new(StringArray::from(vec!["a", "b", "c", "d", "e"])) as Arc<dyn Array>, | ||
| ), | ||
| ( | ||
| "bool_col", | ||
| Arc::new(BooleanArray::from(vec![true, false, true, false, true])) | ||
| as Arc<dyn Array>, | ||
| ), | ||
| ]) | ||
| .unwrap(), | ||
| )); | ||
|
|
||
| // Wrap in FilteredEngineData with all rows selected | ||
| let filtered_data = crate::FilteredEngineData::with_all_rows_selected(engine_data); | ||
|
|
||
| // Write the data | ||
| let file_url = Url::parse("memory:///roundtrip/test.parquet").unwrap(); | ||
| parquet_handler | ||
| .write_parquet_file(file_url.clone(), filtered_data) | ||
| .unwrap(); | ||
|
|
||
| // Read it back | ||
| let path = Path::from_url_path(file_url.path()).unwrap(); | ||
| let reader = ParquetObjectReader::new(store.clone(), path); | ||
| let physical_schema = ParquetRecordBatchStreamBuilder::new(reader) | ||
| .await | ||
| .unwrap() | ||
| .schema() | ||
| .clone(); | ||
|
|
||
| let file_meta = FileMeta { | ||
| location: file_url.clone(), | ||
| last_modified: 0, | ||
| size: 0, | ||
| }; | ||
|
|
||
| let data: Vec<RecordBatch> = parquet_handler | ||
| .read_parquet_files( | ||
| slice::from_ref(&file_meta), | ||
| Arc::new(physical_schema.try_into_kernel().unwrap()), | ||
| None, | ||
| ) | ||
| .unwrap() | ||
| .map(into_record_batch) | ||
| .try_collect() | ||
| .unwrap(); | ||
|
|
||
| // Verify the data | ||
| assert_eq!(data.len(), 1); | ||
| assert_eq!(data[0].num_rows(), 5); | ||
| assert_eq!(data[0].num_columns(), 3); | ||
|
|
||
| // Verify column values | ||
| let int_col = data[0] | ||
| .column(0) | ||
| .as_any() | ||
| .downcast_ref::<Int64Array>() | ||
| .unwrap(); | ||
| assert_eq!(int_col.values(), &[1, 2, 3, 4, 5]); | ||
|
|
||
| let string_col = data[0] | ||
| .column(1) | ||
| .as_any() | ||
| .downcast_ref::<StringArray>() | ||
| .unwrap(); | ||
| assert_eq!(string_col.value(0), "a"); | ||
| assert_eq!(string_col.value(4), "e"); | ||
|
|
||
| let bool_col = data[0] | ||
| .column(2) | ||
| .as_any() | ||
| .downcast_ref::<BooleanArray>() | ||
| .unwrap(); | ||
| assert!(bool_col.value(0)); | ||
| assert!(!bool_col.value(1)); | ||
| } | ||
| } | ||
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.
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.
We should use the async_writer
As they note on that page: object_store provides it’s native implementation of AsyncFileWriter by ParquetObjectWriter.
So you could do something like: