Skip to content

[EPIC] [Parquet] Implement Variant type support in Parquet #6736

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

Open
11 tasks
alamb opened this issue Nov 15, 2024 · 28 comments
Open
11 tasks

[EPIC] [Parquet] Implement Variant type support in Parquet #6736

alamb opened this issue Nov 15, 2024 · 28 comments
Labels
enhancement Any new improvement worthy of a entry in the changelog parquet Changes to the parquet crate

Comments

@alamb
Copy link
Contributor

alamb commented Nov 15, 2024

Is your feature request related to a problem or challenge? Please describe what you are trying to do.
Parquet recently adopted the Variant type from Spark: https://github.com/apache/parquet-format/blob/master/VariantEncoding.md

Details on

Describe the solution you'd like
I would like to implement variant support in parquet-rs

Additional context
I am not sure if any other parquet implementations have implemented this yet / if there are example parquet files. I will attempt to find out

Related Tasks

Related PRs

** Related Community Resources**

@alamb alamb added the enhancement Any new improvement worthy of a entry in the changelog label Nov 15, 2024
@alamb alamb added the parquet Changes to the parquet crate label Nov 15, 2024
@CurtHagenlocher
Copy link
Contributor

There's an implementation in Spark (try here for starters) but when I last looked ~two months ago there was no binary test data; only some round trips via JSON.

@tustvold
Copy link
Contributor

tustvold commented Dec 4, 2024

I do wonder if a precursor to supporting this would be some way to translate / represent the variant data in arrow, whilst there are non-arrow APIs, they'd likely struggle to accommodate this addition, and aren't how the vast majority of people consume parquet data using this crate.

@findepi
Copy link
Member

findepi commented Dec 4, 2024

From arrow perspective, would that be a new DataType, or rather a convention of using DataType::Struct with two Binary fields?

A fully performance variant implementation should be able to leverage file-level column disaggregation (shredding), but I do think this could come as a follow-up to a "normal" Variant type implementation.

@tustvold
Copy link
Contributor

tustvold commented Dec 4, 2024

From arrow perspective, would that be a new DataType, or rather a convention of using DataType::Struct with two Binary fields?

I don't know, I've not really been following the variant proposal close enough to weigh in here. However, my understanding is that shredding is one of the major motivators for this getting added to parquet, as without it you might as well just embed any record format, e.g. Avro. I therefore suspect most use-cases will be at least partially shredded, and the reader will need to handle this case. This is especially true given the variant_value is NULL when the data is shredded, as opposed to say duplicating the content (which would have its own issues TBC), and so we can't just ignore the shredded data.

Unfortunately I can't see an obvious way to be able to represent this sort of semi-structured data within the arrow format without introducing a new DataType that is able to accommodate arrays having the same type, but different child layouts...

TLDR I suspect actioning this will require arrow defining a way to represent semi-structured data...

@findepi
Copy link
Member

findepi commented Dec 4, 2024

There needs to be a way to represent a series of variant values having "no type in common" (variant integer, variant boolean, variant varchar, etc all mixed up). For that some blob-like representation with internal structure seems natural.
Then there should be a way to carry-on the shredded columns without having to put them back into that blob, so yes, one type, different child layouts.
It feels to me that the runtime representation will end up being similar to what is defined in Parquet (https://github.com/apache/parquet-format/blob/master/VariantShredding.md)... so maybe it should be the same representation to provide for an efficient read path.

@findepi
Copy link
Member

findepi commented Dec 4, 2024

When considering what to do in Arrow, we should also keep an eye on the ongoing effort in Iceberg apache/iceberg#10392 (comment)
This could inform some design decisions.
cc @Xuanwo

@alamb
Copy link
Contributor Author

alamb commented Jan 24, 2025

There is now a related proposal being added to paruqet (shredding the variant type:

FWIW I would love to have some rust representation (aka someone who wanted to implement the variant type in the rust parquet decoder). If you are interested there are relevant conversations going on on the parquet mailing list

@alamb
Copy link
Contributor Author

alamb commented Jan 24, 2025

to be clear, we may not be able to merge / really use variants in rust until it gets into Arrow, but we can work out how it would work in parquet (maybe with the non arrow interfaces) first

@CurtHagenlocher
Copy link
Contributor

There's a pure Python implementation in the Spark repo. It's almost standalone, having a dependency only on PySparkValueError.

@alamb
Copy link
Contributor Author

alamb commented Mar 6, 2025

I spent some time listening and thinking about this on the parquet call yesterday: https://lists.apache.org/thread/cnn6264g56jktrwmplz89x8cgkcvr4ql

Note there is a thread on the arrow mailing list about adding variant support in arrow-rs:

(and it looks like @wjones127 made some sort of demo using an extension type in datafusion):

Unfortunately I can't see an obvious way to be able to represent this sort of semi-structured data within the arrow format

What I suggest is that the parquet reader reads variant columns as Binary / LargeBinary with an arrow extension type annotation, which would let downstream projects interpret / read the extension type correctly

I think one challenge will be "how to tell the parquet writer to write / annotate the columns as variant"

Before we can do anything useful with the variant type, we'll need a library to parse / interpret a variant value (aka the equivalent of a JSON parser / set of objects)

@alamb
Copy link
Contributor Author

alamb commented Mar 6, 2025

So the first step as I see it is that someone has to code up / find a Rust implementation for working with variant values. This could be a port or inspired by the ones @CurtHagenlocher 's points at here:

There's a pure Python implementation in the Spark repo.

There's an implementation in Spark (try here for starters)

Once we have such a library, we can then figure out if/how it should be used in the parquet reader/writer directly

@wjones127
Copy link
Member

I haven't had time to work on this recently, but for a rust implementation for working with variant values, anyone should feel free to work off what I had started in the https://github.com/datafusion-contrib/datafusion-functions-variant/tree/main/open-variant repo. There is a core open-variant crate there that's just meant to be reading and writing variant values.

@alamb
Copy link
Contributor Author

alamb commented Mar 6, 2025

@alamb
Copy link
Contributor Author

alamb commented Mar 6, 2025

Since variant is part of the parquet spec now I think the code to interpret could easily belong on the arrow-rs repository and the parquet module. It will be an interesting Rust API design challenge I think to make a really efficient/zero copy decoder

@alamb
Copy link
Contributor Author

alamb commented Mar 8, 2025

Here is a PR to implement variant in C/C++ from @neilechao

@alamb
Copy link
Contributor Author

alamb commented Apr 1, 2025

I have requested example VARIANTs on the mailing list as well:

@adriangb
Copy link
Contributor

adriangb commented Apr 6, 2025

Digging deep into some Spark code I found some pretty enlightening information about how this will actually be encoded into Parquet: apache/spark@3c3d1a6#diff-ca9eeead72220965c7bbd52631f7125d4c1ef22b898e5baec83abc7be9495325

So it seems that apache/datafusion#2581 / apache/datafusion#11745 will ultimately be a blocker for proper support.

I think the things we'll need here are:

  • Ability to project individual struct fields, in particular column -> typed_value -> field_name, for selection and during predicate pushdown pruning
  • Functions that operate on the entire structure and know how to parse the binary metadata/value fields
  • A type that you can declare at the schema level that doesn't force you to exhaustively define the unknown typed fields of the struct
  • Statistics support for nested struct fields

On the DataFusion side I think all we need is something like apache/datafusion#15057 to allow rewriting a filter or projection such as variant_get(col, 'key') = 5 into "col.typed_value.key.typed_value" = 5 on a per-file level if we see from the file schema that a is shredded. Then if all of the above is in place stats filtering, selecting reading of the column for filtering / projection, etc.

@alamb
Copy link
Contributor Author

alamb commented Apr 6, 2025

I have some news I would like to share here -- it seems that @PinkCrow007 has actually been working on a variant implementation in parquet (including support in arrow-rs as an extension type)

Here is an update from Martin Prammer (not sure if he has a github handle)

We've made progress towards implementing the Variant type in both Parquet_rs and Arrow_rs and have prepared a document, shared as a Google doc, that details the overall project and our current status. In summary, our current prototypes are focused on round-tripping binary data between Parquet and Arrow. The Arrow-side Variant is implemented as a CanonicalExtensionType, while the Parquet-side Variant is a LogicalType. If you're interested in looking at the code early, Jiaying's fork is publically available. Our next goal is to implement binary data decoding/encoding to facilitate using Variants as a stand-alone type, which will then allow us to implement Variant shredding. While there's still work to do before we have the basic functionality for a Variant type, we plan to PR the baseline variant and then address shredding.

At this point, it would be helpful for our team to connect to the broader Apache ecosystem's discussion on Variants; Jiaying has already joined the Arrow discord, and we're both happy to join any relevant mailing lists. We're also soliciting existing Variant implementations that we can use to verify our library against.

It seem they also need some example variant data to make faster progress. I will go beg some more from the parquet mailing list

It is very exciting to see the momentum picking up

@alamb
Copy link
Contributor Author

alamb commented Apr 6, 2025

@alamb
Copy link
Contributor Author

alamb commented Apr 7, 2025

I got a response from @cashmand ❤ and I filed an issue in the parquet-testing repo to track the work to add examples

@adriangb
Copy link
Contributor

adriangb commented Apr 7, 2025

Here is an update from Martin Prammer (not sure if he has a github handle)

Seems like @mprammer does 😄

@alamb
Copy link
Contributor Author

alamb commented Apr 8, 2025

I just had a discussion with @PinkCrow007 and @PinkCrow007, as I understand it the next steps will be:

  1. Create a draft / work in progress PR that we can start reviewing / providing feedback on based on the fork: main...PinkCrow007:arrow-rs:variant-clean.

The eventual goal will be to break it up and start merging it in as pieces:

  1. Support for reading binary (&[u8]) as variants (accessing via fields, etc) -- what appears to be in arrow-variant/src/encoder in the fork
  2. Support for reading/writing to parquet
  3. SUpport for shredding, etc

It is going to be so great

@alamb
Copy link
Contributor Author

alamb commented Apr 8, 2025

I took a quick skim through the code in main...PinkCrow007:arrow-rs:variant-clean and I found it easy to understand and well structured. I am very much looking forward to the PR

@alamb alamb changed the title [Parquet] Implement Variant type support in Parquet [EPIC] [Parquet] Implement Variant type support in Parquet Apr 11, 2025
@alamb
Copy link
Contributor Author

alamb commented Apr 13, 2025

Update in case anyone didn't see it: @PinkCrow007 has created a draft PR for comment:

Related: C/C++ implementation in arrow:

@alamb
Copy link
Contributor Author

alamb commented Apr 13, 2025

I have been studying the variant spec, and various implementation. It seems variant support to/from json is quite well covered. There are things in the spec (like Time, for example) that are not well represented/implemented in open source spark. I'll. definitely focus on the JSON stuff first

@alamb
Copy link
Contributor Author

alamb commented Apr 18, 2025

I spent a while today writing up some first sugested steps and linked them to the ticket

It would be great if someone wanted to take a crack at

I think that will unblock a lot of the rest of what is going on

I am out the rest of the week but hopefully I can check email occasionally

@alamb
Copy link
Contributor Author

alamb commented Apr 28, 2025

@findepi -- I heard today you may be working on variant support as well. I wonder if you have any thoughts about the above plan (or perhaps already have it implemented and would be willing to share 😆 )

@alamb
Copy link
Contributor Author

alamb commented May 8, 2025

Status report (also reported to the parquet mailing list): https://lists.apache.org/thread/dy22njos6c0wbo82s377wvbobbd7y6lx

I am pretty stoked to report progress on a Rust Variant implementation (see epic here0).

  1. We have added example binary data in parquet-testing1 (Thanks Micah for the review and merge)
  2. Jiaying's prototype 2 is looking good and we are preparing to start merging it in pieces
  3. I have created a PR to add a parquet-variant crate to arrow-rs3 (looking for a review 🎣)

Once we have the parquet-variant crate merged, I expect a series of PRs that incrementally adds support for different parts of the Variant specification. Once this is done, we will move on to integrating it into the parquet decoder.

Exciting times,
Andrew

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
enhancement Any new improvement worthy of a entry in the changelog parquet Changes to the parquet crate
Projects
None yet
Development

No branches or pull requests

6 participants