Skip to content

Conversation

@yabola
Copy link
Contributor

@yabola yabola commented Feb 24, 2023

In parquet pageV1(Spark use parquet v1 default), even all pages of a column are encoded by dictionary, it will still generate BloomFilter. Actually it is unnecessary and it costs time and occupies storage. Parquet pageV2 doesn't have this problem.

@yabola
Copy link
Contributor Author

yabola commented Feb 24, 2023

@wgtmac @gszadovszky Please take a look, thank you~
And I will update PR to skip bloomfilter when all pages are encoded in dictionary (because there are already many wrong parquet v1 files generated)

Copy link
Member

@wgtmac wgtmac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

cc @gszadovszky

@Parameterized.Parameters(name = "Run {index}: parquet {1}")
public static Collection<Object[]> params() {
return Arrays.asList(
new Object[]{FILE_V1, "pageV1"},
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: the name is a little bit confusing. file format version is not same as data page version.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks, I modified the description

@yabola yabola changed the title PARQUET-2251 Avoid generating Bloomfilter when all pages of a column are encoded by dictionary in parquet pageV1 PARQUET-2251 Avoid generating Bloomfilter when all pages of a column are encoded by dictionary in parquet v1 Feb 24, 2023
Copy link
Contributor

@gszadovszky gszadovszky left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks, @yabola, @wgtmac.

@gszadovszky gszadovszky merged commit 4e9e79c into apache:master Feb 27, 2023
@yabola
Copy link
Contributor Author

yabola commented Feb 27, 2023

@wgtmac @gszadovszky Thank you for your review and help

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

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants