steveloughran commented on PR #3128: URL: https://github.com/apache/parquet-java/pull/3128#issuecomment-2607114605
I should highlight that there is some WiP from amazon to add a parquet-aware input stream: https://issues.apache.org/jira/browse/HADOOP-19363 This parses the footer, tries to prefetch column chunks based on what has just been read. Their benchmarks show ~the same speedup against unvectored IO. They did't benchmark against vector IO, which is a pity. In its draft form it doesn't support vector IO, just classic read()s. However, I plan to add it by pulling the relevant code up, and integrating it with the new stream such that * entire range is cached in memory: return that. * entire range is being actively prefetched: add to whatever callback is going to be invoked at the end of the ongoing read * not in cache, not being fetched, or only partially present/retrieved. keep on list of ranges to fetch as usual After this pruning, do the merge and fetch as before. _do not cache these results_ I think I'll need to look at what the current algorithms for prefetching and caching in operating systems is. Stonebraker's classic "Operating System Support for Database Management" is fairly old, after all. (If anyone hasn't read this yet -do it) Going to be fun testing all this. I'm actually adding format-test module for hadoop to validate the new code against the most recent stable releases of parquet, iceberg, etc so we can do regression tests against the actual object stores. This is test only -no need to worry about a loop in the build. It's just that given the complexity of the new stream, and at lot of being in the AWS SDK, I want that testing on every SDK update. I need to make sure that malformed files are handled correctly too... -- 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: issues-unsubscr...@parquet.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@parquet.apache.org For additional commands, e-mail: issues-h...@parquet.apache.org