-
Notifications
You must be signed in to change notification settings - Fork 1.2k
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
Expose parquet reader settings using normal DataFusion ConfigOptions
#3822
Conversation
ConfigOptions
@@ -109,6 +111,13 @@ async fn main() -> Result<()> { | |||
Ok(()) | |||
} | |||
|
|||
#[derive(Debug, Clone)] | |||
struct ParquetScanOptions { |
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 was replicated for the benchmark code as I felt such a struct was the easiest to understand for this matrix strategy
4096, | ||
), | ||
ConfigDefinition::new_string( |
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 moved this to be with the other settings
/// Create new ConfigOptions struct, taking values from | ||
/// environment variables where possible. | ||
/// | ||
/// For example, setting `DATAFUSION_EXECUTION_BATCH_SIZE` will |
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 will add some documentation about this to the datafusion-cli docs as I couldn't find it when I was looking
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.
@@ -69,43 +72,6 @@ use parquet::file::{ | |||
}; | |||
use parquet::schema::types::ColumnDescriptor; | |||
|
|||
#[derive(Debug, Clone, Default)] | |||
/// Specify options for the parquet scan | |||
pub struct ParquetScanOptions { |
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.
The key change to this PR is removing this structure and instead reading the values from a ConfigOptions
that is threaded down.
You can see in this PR there is already a structure for configuring parquet reading (ParquetReadOptions
) so I actually think this will make the code less confusing to work with going forward.
/// `ParquetRecordBatchStream`. These filters are applied by the | ||
/// parquet decoder to skip unecessairly decoding other columns | ||
/// which would not pass the predicate. Defaults to false | ||
pub fn with_pushdown_filters(self, pushdown_filters: bool) -> Self { |
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 just uses the slightly messier config API to get/set the settings
let reorder_predicates = self.scan_options.reorder_predicates; | ||
let pushdown_filters = self.scan_options.pushdown_filters; | ||
let enable_page_index = self.scan_options.enable_page_index; | ||
let reorder_predicates = self.reorder_filters; |
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 also took the opportunity to change to consistently use the word filters
rather than filters
and predciates
.with_pushdown_filters(true) | ||
.with_reorder_predicates(true), | ||
); | ||
parquet_exec = parquet_exec |
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.
Here is a good illustration of how the API changes (I think for the better)
@@ -160,10 +170,12 @@ pub struct ParquetReadOptions<'a> { | |||
pub table_partition_cols: Vec<String>, | |||
/// Should DataFusion parquet reader use the predicate to prune data, | |||
/// overridden by value on execution::context::SessionConfig | |||
// TODO move this into ConfigOptions |
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 will do this as a follow on PR
Nice! I will review this this evening or tomorrow depending on how the day goes |
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.
Had a few comments about why we need a shareable ConfigOptions
down at the exec layer but overall this is a great improvement
/// minimize the cost of filter evaluation by reordering the | ||
/// predicate [`Expr`]s. If false, the predicates are applied in | ||
/// the same order as specified in the query. Defaults to false. | ||
pub fn with_reorder_filters(self, reorder_filters: bool) -> Self { |
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.
See above, wrapping the options in a Arc<RwLock<_>>
seems strange since this is already essentially an owned value.
@@ -698,6 +699,7 @@ mod tests { | |||
projection, | |||
statistics, | |||
table_partition_cols, | |||
config_options: ConfigOptions::new().into_shareable(), |
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.
Not sure I understand why we need the into_shareable
here. Seems like this should just be an owned ConfigOptions
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 agree it is non obvious -- ConfigOptions
was changed to be shareable by @waitingkuo in #3455
Basically the usecase there was so that the configuration was owned by SessionContext
but other parts could read it if necessary -- specifically, information_schema.df_settings
/ SHOW ALL
initially. This PR extends the concept so that the settings can be read by the parquet reader
What would you think about moving the mutability handling into ConfigOption
so this looks like
config_options: ConfigOptions::new(),
That would hide the details of shared ownership more nicely I think
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.
Yeah, that makes sense. If ConfigOptions
is meant to be shareable then it can just hold a Arc<RwLock<HashMap<_>>
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.
Will do as a follow on PR
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.
Filed #3886
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.
PR to clean up: #3909
I plan to merge this after it passes CI to keep the process going. I really like the |
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.
LGTM. Thanks @alamb
cc6c3db
to
900e15f
Compare
Which issue does this PR close?
Re #3821
Rationale for this change
I want to test out the parquet filter pushdown on real datasets using datafusion-cli so we can enable it by default -- #3463
I also want an easy way to disable the feature if users find they are getting wrong results
I want to be able to do so via
datafusion-cli
like:And then set them like:
What changes are included in this PR?
ConfigOptions
ConfigOptions
down into the FileScanConfigParquetScanOptions
in favor of these new configs (will comment on the rationale here)Are there any user-facing changes?
YES: If you used
ParquetScanOptions
(which I know @thinkharderdev does) the API has changed.Also, the settings are now visible in the user level documentation