-
Notifications
You must be signed in to change notification settings - Fork 2.4k
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
[SUPPORT] parquet bloom filters not supported by hudi #7117
Comments
hi @nsivabalan bloom at read time is an useful feature for read performance. thought ? |
Hudi is able to benefit from parquet files written with blooms. (tested by replacing the hudi parquet files with the vanilla spark's one, and it hudi datasource triggers the bloom). Digging the source code, I guess the reason blooms are not taken in consideration is in the hudi's parquetWriter wrapper . It then calls the parquetWriter public constructor which has very limited parquet feature support. There is a more complete constructor but sadly it's access is limited to package. Accessing to package constructor can be done by changing the A better option would be parquet provides more suitable constructors. Also we could make use of the builder and there is some pointers here |
@danny0405 thought ? |
2 questions:
|
as for 1 please see the OP and let me know if this is still unclear. as for 2, spark leverages parquet library to writes blooms. It just pass the properties from the user with the datasource api |
Yeah, please elaborate more about 1, and for 2, can Hudi also take the similiar manner for writers? |
hudi should support parquet vanilla bloom filters, because this is a standard optimization method supported by every query engines using parquet 1.12 and above. Moreover hudi does not provide such optimization method. Hudi blooms are not used for select queries. Hudi blooms are only useful for update operations. Providing vanilla parquet bloom support to hudi would allow an other set of optimization (such z-order, parquet stats) for almost free. as for 2, providing bloom filters to hudi is just passing the configurations to parquet. But after analysis, it likely needs a small refactoring of the way we configure parquet writers. (by stopping using deprecated ParquetWriter constructor and use the builder method which support all parquet configurations) |
more details: |
I'm okay with that. |
How does
increment when
|
that's a good point. I don't know, I found that code in the spark tests. The point is it does increment ! |
#10278 I am working on the FileGroup Reader for Hudi 1.0 and that test was failing but if I change it to accu.add(1) then it works. So that's why I'm asking. I don't want to break parquet bloom filters. |
Okay will double check thanks for reaching |
@parisni update on this. I am working on getting vectorized reader enabled in more scenarios with the new fg reader and now that test works again |
hudi 0.12.1
spark 3.2.1
hudi has its own bloom implementation used mainly for fast lookup on the hudi key. Up to 0.11 hudi allow to store them in the metadata table, and also add bloom on several columns. So far those bloom are not use at read time.
Spark leverage parquet bloom filters at write time. They are then used at read time to skip files. Those bloom can improve queries a lot. But so far, hudi don't support them :
Basic parquet table => bloom works as expected: the accumulator is not used when filtered values are outside df values, but incremented when values matches
Hudi parquet table => bloom works NOT as expected: the accumulator is incremented when match and no match
The text was updated successfully, but these errors were encountered: