-
Notifications
You must be signed in to change notification settings - Fork 2k
Add FileStreamBuilder for creating FileStreams
#21261
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 7 commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
53aa738
Add `FileStreamBuilder` for creating FileStreams
alamb ca8e289
Deprecate
alamb f96f85f
modularize
alamb 0bbaf3a
Simplify
alamb 844b39a
simplify tests
alamb c14fd8f
moar simple
alamb fed2f18
cleanup
alamb 36da15c
Merge branch 'main' into alamb/file_stream_builder
alamb 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
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,108 @@ | ||
| // 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::sync::Arc; | ||
|
|
||
| use crate::file_scan_config::FileScanConfig; | ||
| use datafusion_common::{Result, internal_err}; | ||
| use datafusion_physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet}; | ||
|
|
||
| use super::{FileOpener, FileStream, FileStreamMetrics, FileStreamState, OnError}; | ||
|
|
||
| /// Builder for constructing a [`FileStream`]. | ||
| pub struct FileStreamBuilder<'a> { | ||
| config: &'a FileScanConfig, | ||
| partition: Option<usize>, | ||
| file_opener: Option<Arc<dyn FileOpener>>, | ||
| metrics: Option<&'a ExecutionPlanMetricsSet>, | ||
| on_error: OnError, | ||
| } | ||
|
|
||
| impl<'a> FileStreamBuilder<'a> { | ||
| /// Create a new builder. | ||
| pub fn new(config: &'a FileScanConfig) -> Self { | ||
| Self { | ||
| config, | ||
| partition: None, | ||
| file_opener: None, | ||
| metrics: None, | ||
| on_error: OnError::Fail, | ||
| } | ||
| } | ||
|
|
||
| /// Configure the partition to scan. | ||
| pub fn with_partition(mut self, partition: usize) -> Self { | ||
| self.partition = Some(partition); | ||
| self | ||
| } | ||
|
|
||
| /// Configure the [`FileOpener`] used to open files. | ||
| pub fn with_file_opener(mut self, file_opener: Arc<dyn FileOpener>) -> Self { | ||
| self.file_opener = Some(file_opener); | ||
| self | ||
| } | ||
|
|
||
| /// Configure the metrics set used by the stream. | ||
| pub fn with_metrics(mut self, metrics: &'a ExecutionPlanMetricsSet) -> Self { | ||
| self.metrics = Some(metrics); | ||
| self | ||
| } | ||
|
|
||
| /// Configure the behavior when opening or scanning a file fails. | ||
| pub fn with_on_error(mut self, on_error: OnError) -> Self { | ||
| self.on_error = on_error; | ||
| self | ||
| } | ||
|
|
||
| /// Build the configured [`FileStream`]. | ||
| pub fn build(self) -> Result<FileStream> { | ||
| let Self { | ||
| config, | ||
| partition, | ||
| file_opener, | ||
| metrics, | ||
| on_error, | ||
| } = self; | ||
|
|
||
| let Some(partition) = partition else { | ||
| return internal_err!("FileStreamBuilder missing required partition"); | ||
| }; | ||
| let Some(file_opener) = file_opener else { | ||
| return internal_err!("FileStreamBuilder missing required file_opener"); | ||
| }; | ||
| let Some(metrics) = metrics else { | ||
| return internal_err!("FileStreamBuilder missing required metrics"); | ||
| }; | ||
| let projected_schema = config.projected_schema()?; | ||
| let Some(file_group) = config.file_groups.get(partition).cloned() else { | ||
| return internal_err!( | ||
| "FileStreamBuilder invalid partition index: {partition}" | ||
| ); | ||
| }; | ||
|
|
||
| Ok(FileStream { | ||
| file_iter: file_group.into_inner().into_iter().collect(), | ||
| projected_schema, | ||
| remain: config.limit, | ||
| file_opener, | ||
| state: FileStreamState::Idle, | ||
| file_stream_metrics: FileStreamMetrics::new(metrics, partition), | ||
| baseline_metrics: BaselineMetrics::new(metrics, partition), | ||
| on_error, | ||
| }) | ||
| } | ||
| } |
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 |
|---|---|---|
|
|
@@ -21,6 +21,8 @@ | |
| //! Note: Most traits here need to be marked `Sync + Send` to be | ||
| //! compliant with the `SendableRecordBatchStream` trait. | ||
|
|
||
| mod builder; | ||
|
|
||
| use std::collections::VecDeque; | ||
| use std::pin::Pin; | ||
| use std::sync::Arc; | ||
|
|
@@ -29,7 +31,7 @@ use std::task::{Context, Poll}; | |
| use crate::PartitionedFile; | ||
| use crate::file_scan_config::FileScanConfig; | ||
| use arrow::datatypes::SchemaRef; | ||
| use datafusion_common::error::Result; | ||
| use datafusion_common::Result; | ||
| use datafusion_execution::RecordBatchStream; | ||
| use datafusion_physical_plan::metrics::{ | ||
| BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricCategory, Time, | ||
|
|
@@ -42,6 +44,8 @@ use futures::future::BoxFuture; | |
| use futures::stream::BoxStream; | ||
| use futures::{FutureExt as _, Stream, StreamExt as _, ready}; | ||
|
|
||
| pub use builder::FileStreamBuilder; | ||
|
|
||
| /// A stream that iterates record batch by record batch, file over file. | ||
| pub struct FileStream { | ||
| /// An iterator over input files. | ||
|
|
@@ -66,26 +70,18 @@ pub struct FileStream { | |
|
|
||
| impl FileStream { | ||
| /// Create a new `FileStream` using the give `FileOpener` to scan underlying files | ||
| #[deprecated(since = "54.0.0", note = "Use FileStreamBuilder instead")] | ||
| pub fn new( | ||
| config: &FileScanConfig, | ||
| partition: usize, | ||
| file_opener: Arc<dyn FileOpener>, | ||
| metrics: &ExecutionPlanMetricsSet, | ||
| ) -> Result<Self> { | ||
| let projected_schema = config.projected_schema()?; | ||
|
|
||
| let file_group = config.file_groups[partition].clone(); | ||
|
|
||
| Ok(Self { | ||
| file_iter: file_group.into_inner().into_iter().collect(), | ||
| projected_schema, | ||
| remain: config.limit, | ||
| file_opener, | ||
| state: FileStreamState::Idle, | ||
| file_stream_metrics: FileStreamMetrics::new(metrics, partition), | ||
| baseline_metrics: BaselineMetrics::new(metrics, partition), | ||
| on_error: OnError::Fail, | ||
| }) | ||
| FileStreamBuilder::new(config) | ||
| .with_partition(partition) | ||
| .with_file_opener(file_opener) | ||
| .with_metrics(metrics) | ||
| .build() | ||
| } | ||
|
|
||
| /// Specify the behavior when an error occurs opening or scanning a file | ||
|
|
@@ -400,17 +396,17 @@ impl FileStreamMetrics { | |
|
|
||
| #[cfg(test)] | ||
| mod tests { | ||
| use crate::PartitionedFile; | ||
| use crate::file_scan_config::FileScanConfigBuilder; | ||
| use crate::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; | ||
| use crate::tests::make_partition; | ||
| use crate::{PartitionedFile, TableSchema}; | ||
| use datafusion_common::error::Result; | ||
| use datafusion_execution::object_store::ObjectStoreUrl; | ||
| use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; | ||
| use futures::{FutureExt as _, StreamExt as _}; | ||
| use std::sync::Arc; | ||
| use std::sync::atomic::{AtomicUsize, Ordering}; | ||
|
|
||
| use crate::file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; | ||
| use crate::file_stream::{FileOpenFuture, FileOpener, FileStreamBuilder, OnError}; | ||
| use crate::test_util::MockSource; | ||
| use arrow::array::RecordBatch; | ||
| use arrow::datatypes::Schema; | ||
|
|
@@ -530,7 +526,7 @@ mod tests { | |
|
|
||
| let on_error = self.on_error; | ||
|
|
||
| let table_schema = crate::table_schema::TableSchema::new(file_schema, vec![]); | ||
| let table_schema = TableSchema::new(file_schema, vec![]); | ||
| let config = FileScanConfigBuilder::new( | ||
| ObjectStoreUrl::parse("test:///").unwrap(), | ||
| Arc::new(MockSource::new(table_schema)), | ||
|
|
@@ -539,10 +535,12 @@ mod tests { | |
| .with_limit(self.limit) | ||
| .build(); | ||
| let metrics_set = ExecutionPlanMetricsSet::new(); | ||
| let file_stream = | ||
| FileStream::new(&config, 0, Arc::new(self.opener), &metrics_set) | ||
| .unwrap() | ||
| .with_on_error(on_error); | ||
|
Contributor
Author
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. FileStream was already starting to grow a builder interface as it gained features, so let's just make it directly |
||
| let file_stream = FileStreamBuilder::new(&config) | ||
| .with_partition(0) | ||
| .with_file_opener(Arc::new(self.opener)) | ||
| .with_metrics(&metrics_set) | ||
| .with_on_error(on_error) | ||
| .build()?; | ||
|
|
||
| file_stream | ||
| .collect::<Vec<_>>() | ||
|
|
@@ -563,6 +561,23 @@ mod tests { | |
| .expect("error executing stream") | ||
| } | ||
|
|
||
| /// Create the smallest valid file scan config for builder validation tests. | ||
| fn builder_test_config() -> FileScanConfig { | ||
| let table_schema = TableSchema::new(Arc::new(Schema::empty()), vec![]); | ||
| FileScanConfigBuilder::new( | ||
| ObjectStoreUrl::parse("test:///").unwrap(), | ||
| Arc::new(MockSource::new(table_schema)), | ||
| ) | ||
| .with_file(PartitionedFile::new("mock_file", 10)) | ||
| .build() | ||
| } | ||
|
|
||
| /// Convenience helper to keep builder error assertions focused on the | ||
| /// specific missing or invalid input under test. | ||
| fn builder_error(builder: FileStreamBuilder<'_>) -> String { | ||
| builder.build().err().unwrap().to_string() | ||
| } | ||
|
|
||
| #[tokio::test] | ||
| async fn on_error_opening() -> Result<()> { | ||
| let batches = FileStreamTest::new() | ||
|
|
@@ -857,4 +872,36 @@ mod tests { | |
|
|
||
| Ok(()) | ||
| } | ||
|
|
||
| #[test] | ||
| fn builder_requires_partition_file_opener_and_metrics() { | ||
| let config = builder_test_config(); | ||
|
|
||
| let err = builder_error(FileStreamBuilder::new(&config)); | ||
| assert!(err.contains("FileStreamBuilder missing required partition")); | ||
|
|
||
| let err = builder_error(FileStreamBuilder::new(&config).with_partition(0)); | ||
| assert!(err.contains("FileStreamBuilder missing required file_opener")); | ||
|
|
||
| let err = builder_error( | ||
| FileStreamBuilder::new(&config) | ||
| .with_partition(0) | ||
| .with_file_opener(Arc::new(TestOpener::default())), | ||
| ); | ||
| assert!(err.contains("FileStreamBuilder missing required metrics")); | ||
| } | ||
|
|
||
| #[test] | ||
| fn builder_errors_on_invalid_partition() { | ||
| let config = builder_test_config(); | ||
| let metrics = ExecutionPlanMetricsSet::new(); | ||
|
|
||
| let err = builder_error( | ||
| FileStreamBuilder::new(&config) | ||
| .with_partition(1) | ||
| .with_file_opener(Arc::new(TestOpener::default())) | ||
| .with_metrics(&metrics), | ||
| ); | ||
| assert!(err.contains("FileStreamBuilder invalid partition index: 1")); | ||
| } | ||
| } | ||
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.
this illustrates the new API pretty well -- instead of a pile of arguments to
FileStream::newthere is nowFileStreamBuilder