Open blakesmith opened 1 month ago
Just going to highlight that hadoop 3.4.0 + supports scatter/gather IO for local, s3 connections. If you do go with this you should benchmark the work and consider pulling up the vector IO stuff into the base parquet code, calling down to hadoop as appropriate.
We see a 30% improvement in TPC/DS benchmarks with that option against s3; local IO shows some as we can do parallelized NIO reads.
Describe the enhancement requested
Hi there!
In an effort to reduce our coupling to Hadoop, and slim down our shaded Jar sizes, we transitioned all our Parquet reading / writing to using the non-Hadoop public interfaces. We leveraged Java NIO interfaces for reading / writing I/O, as detailed in this blog post here: https://blakesmith.me/2024/10/05/how-to-use-parquet-java-without-hadoop.html
The problem we hit: The current
LocalInputFile
implementation falls back to Path#toFile and RandomAccessFile calls for I/O operations. Once we plug in otherjava.nio.file.FileSystem
implementations (Such as using AWS' Java NIO SPI implementation), theLocalInputFile
implementation doesn't work anymore.We've created our own implementation of
LocalInputFile
calledNioInputFile
, that implements the interface, but exclusively using Java NIO operations, instead of falling back to older Java file APIs. You can see the code in the above linked blog post, and also in this gist here: https://gist.github.com/blakesmith/5c08a2504c08c96f9fb62658ec6c9b28A few questions, before I start sending patches:
LocalInputFile
, or addNioInputFile
as another implementation ofLocalInputFile
? Note that in our case, we're doing buffered I/O to S3 with this implementation, so it's not technically a "local" file, either way, just one that talks to the Java NIO abstractions.Of note is that
LocalOutputFile
is working just fine with Java NIO operations, even when we're doing remote S3 I/O.Thanks you so much! Parquet has been transformative for us, and we're excited to help drive more adoption however we can.
Component(s)
Core