Skip to content
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

Pure Java NIO LocalInputFile Implementation #3028

Open
blakesmith opened this issue Oct 7, 2024 · 1 comment
Open

Pure Java NIO LocalInputFile Implementation #3028

blakesmith opened this issue Oct 7, 2024 · 1 comment

Comments

@blakesmith
Copy link

blakesmith commented Oct 7, 2024

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 other java.nio.file.FileSystem implementations (Such as using AWS' Java NIO SPI implementation), the LocalInputFile implementation doesn't work anymore.

We've created our own implementation of LocalInputFile called NioInputFile, 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/5c08a2504c08c96f9fb62658ec6c9b28

A few questions, before I start sending patches:

  1. Is this alternative implementation something that folks are interested in incorporating upstream into Parquet itself?
  2. If so, would we prefer to update LocalInputFile, or add NioInputFile as another implementation of LocalInputFile? 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

@steveloughran
Copy link
Contributor

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.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

2 participants