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
ARROW-11014: [Rust] [DataFusion] Use correct statistics for ParquetExec #8992
Conversation
@seddonm1 fyi |
Codecov Report
@@ Coverage Diff @@
## master #8992 +/- ##
==========================================
+ Coverage 82.64% 82.66% +0.01%
==========================================
Files 200 200
Lines 49730 49798 +68
==========================================
+ Hits 41098 41164 +66
- Misses 8632 8634 +2
Continue to review full report at Codecov.
|
Thanks. Good find. 🤦 by me |
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 wonder if there is some way to test this code -- I also remember being confused about the fact that ParquetExec
can actually take a directory full of files rather than a single one.
use async_trait::async_trait; | ||
use futures::stream::Stream; | ||
|
||
/// Execution plan for scanning a Parquet file | ||
/// Execution plan for scanning one or more Parquet files |
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.
👍
let mut total_byte_size = 0; | ||
for file in &filenames { | ||
let file = File::open(file)?; | ||
let file_reader = Arc::new(SerializedFileReader::new(file)?); |
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.
It probably doesn't matter but we are creating arrow_reader
s several times for the same file -- like here we create them just to read metadata, and then right below we (re)open the first one again to read the schema. And then we open them again to actually read data...
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 is a good point. I've pushed another change here to collect unique schemas during the scan of the files to avoid the separate read. This now also implements a check to make sure the schemas are the same. I have wasted time in the past tracking down issues due to incompatible files. I added a reference to the issue for implementing schema merging, which would be a nice addition.
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've gone a little further and introduced a ParquetPartition
struct to make things more explicit about how partitioning works and added references to related issues for changing the partitioning strategy. I also improved an error message and added more documentation.
Yes, tests are definitely lacking here. I will take this on as a follow-up task for the release: https://issues.apache.org/jira/browse/ARROW-11020 |
@alamb @jorgecarleitao I got a bit carried away with some other improvements in this PR but I am going to stop now. I filed a follow-up issue to add more comprehensive tests before we release 3.0.0 |
Also @Dandandan this starts to introduce some per-partition stats now |
Makes sense. Maybe we can reuse the summing stats for all partitions for different source types if it gets more complex. |
// build a list of Parquet partitions with statistics and gather all unique schemas | ||
// used in this data set | ||
let mut schemas: Vec<Schema> = vec![]; | ||
let mut partitions = vec![]; |
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.
Could use with_capacity
} | ||
let statistics = Statistics { | ||
num_rows: if num_rows == 0 { | ||
None |
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.
Why map rows and byte size to None here and not Some(0)
?
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 think if the result is 0, it is best to know that there are 0 records/bytes.
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.
Yes, that was a bit sloppy. This is now fixed.
pub fn new( | ||
filenames: Vec<String>, | ||
partitions: Vec<ParquetPartition>, |
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 thought that part of the reason for allowing a list of files was to support similar behavior to DeltaLake where an external file contains a list of filenames representing a version of data rather than grouping them by directory structure?
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.
Interesting. I was not aware of this use case. Perhaps we need a specific constructor for that use case. I'll take a look.
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.
It may be premature to support that use case anyway until more of the core engine works.
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 pushed a change so there are now try_from_path
and try_from_files
constructors
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.
Looks good 👍
1 comment that I think should be resolved. The rest looks good, great that this is fixed. |
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.
Nice
num_rows: Some(num_rows as usize), | ||
total_byte_size: Some(total_byte_size as usize), | ||
}; | ||
partitions.push(ParquetPartition { |
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.
👍
ParquetExec represents multiple files but we were calculating statistics based on the first file. I stumbled across this when working on https://issues.apache.org/jira/browse/ARROW-10995 Closes apache#8992 from andygrove/ARROW-11014 Authored-by: Andy Grove <andygrove73@gmail.com> Signed-off-by: Andy Grove <andygrove73@gmail.com>
ParquetExec represents multiple files but we were calculating statistics based on the first file.
I stumbled across this when working on https://issues.apache.org/jira/browse/ARROW-10995