pjmore commented on a change in pull request #2000:
URL: https://github.com/apache/arrow-datafusion/pull/2000#discussion_r828493446
##########
File path: datafusion/src/physical_plan/file_format/parquet.rs
##########
@@ -236,32 +237,56 @@ impl ExecutionPlan for ParquetExec {
let adapter = SchemaAdapter::new(self.base_config.file_schema.clone());
- let join_handle = task::spawn_blocking(move || {
- if let Err(e) = read_partition(
- object_store.as_ref(),
- adapter,
- partition_index,
- &partition,
- metrics,
- &projection,
- &pruning_predicate,
- batch_size,
- response_tx.clone(),
- limit,
- partition_col_proj,
- ) {
- println!(
+ let join_handle = if projection.is_empty() {
+ task::spawn_blocking(move || {
+ if let Err(e) = read_partition_no_file_columns(
+ object_store.as_ref(),
+ &partition,
+ batch_size,
+ response_tx.clone(),
Review comment:
Row group level filtering is not required. Since only partition columns
are queried there will be no filters which will operate on the columns
contained within the parquet file. The file list will already be pruned due to
the listing logic in ListingTable. Similarly the ParquetFileMetrics only
contain metrics about the evaluation of row group predicates evaluation and
filtering I could add the metrics back in but they won't be used for anything.
```
struct ParquetFileMetrics {
/// Number of times the predicate could not be evaluated
pub predicate_evaluation_errors: metrics::Count,
/// Number of row groups pruned using
pub row_groups_pruned: metrics::Count,
}
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]