-
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
Improve parquet ListingTable speed with parquet metadata (short clickbench queries) #11719
Comments
Some ideas about solving
Plan to try it today. See one simple thing is, refactor the pub struct StatisticsInner {
/// The number of table rows.
pub num_rows: Precision<usize>,
/// Total bytes of the table rows.
pub total_byte_size: Precision<usize>,
/// Statistics on a column level. It contains a [`ColumnStatistics`] for
/// each field in the schema of the table to which the [`Statistics`] refer.
pub column_statistics: Vec<ColumnStatistics>,
}
pub struct Statistics {
inner: Arc<StatisticsInner>,
} And the clone of |
take |
That would be a very interesting experiment to try |
Based on the detail profile about q0 in clickbench as following, maybe the optimization work can be divided into three parts:
Trying the first possible optimization now. |
#11802 is very nice 👌 It would be fascinating to know what the flamegraph looks like after that PR (aka what are next highest bottleneck) |
😄 I guess they will be the |
Is your feature request related to a problem or challenge?
I spent some time looking at the ClickBench results with DataFusion 40.0.0
#11567 (comment) (thanks @pmcgleenon 🙏 )
Specifically, I looked into how we could make some of the already fast queries on the the partitioned dataset faster. Unsurprisingly, for the really fast queries the query time is actually dominated by parquet metadata analysis and DataFusion statistics creation.
For example
ClickBench Q0
To reproduce, run:
cd datafusion cargo run --release --bin dfbench -- clickbench --iterations 100 --path benchmarks/data/hits_partitioned --query 0
I profiled this using Instruments. Here are some annotated screenshots
Some of my take aways are
Describe the solution you'd like
If would be cool to make these queries faster by reducing the per file metadata handling overhead (e.g. don't read the metadata more than once and figure out some way to make statistics handling more efficient)
Describe alternatives you've considered
Note this project isn't broken down into tasks yet
I think @Ted-Jiang did some work way back to cache parquet metaddata
Additional context
No response
The text was updated successfully, but these errors were encountered: