-
Notifications
You must be signed in to change notification settings - Fork 1.5k
GH-3123: Omit level histogram for some max levels #3124
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -67,8 +67,16 @@ public static class Builder { | |
| private Builder(PrimitiveType type, int maxRepetitionLevel, int maxDefinitionLevel) { | ||
| this.type = type; | ||
| this.unencodedByteArrayDataBytes = 0L; | ||
| repetitionLevelHistogram = new long[maxRepetitionLevel + 1]; | ||
| definitionLevelHistogram = new long[maxDefinitionLevel + 1]; | ||
| if (maxRepetitionLevel > 0) { | ||
| repetitionLevelHistogram = new long[maxRepetitionLevel + 1]; | ||
| } else { | ||
| repetitionLevelHistogram = new long[0]; // omitted | ||
| } | ||
| if (maxDefinitionLevel > 1) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm curious why you've chosen 1 here, when other implementations use 0. Is it so parquet-java will implement the spec exactly or are you thinking to minimize metadata size? Not a criticism, just genuinely curious.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is pretty easy to implement here and downstream consumers have to deal with omitted case anyway. By doing this, we can produce such kind of example files now. |
||
| definitionLevelHistogram = new long[maxDefinitionLevel + 1]; | ||
| } else { | ||
| definitionLevelHistogram = new long[0]; // omitted | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -79,8 +87,12 @@ private Builder(PrimitiveType type, int maxRepetitionLevel, int maxDefinitionLev | |
| * @param definitionLevel definition level of the value | ||
| */ | ||
| public void add(int repetitionLevel, int definitionLevel) { | ||
| repetitionLevelHistogram[repetitionLevel]++; | ||
| definitionLevelHistogram[definitionLevel]++; | ||
| if (repetitionLevelHistogram.length > 0) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do we have benchmarks for this class? I'm curious if there is a trade-off of always incrementing here and avoiding writing at the end (I suppose JVM might be smart enough to remove the conditionals so I'm fine keeping it as is).
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm not a Java expert so I prefer to keep the current impl just in case. |
||
| repetitionLevelHistogram[repetitionLevel]++; | ||
| } | ||
| if (definitionLevelHistogram.length > 0) { | ||
| definitionLevelHistogram[definitionLevel]++; | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -160,16 +172,29 @@ public void mergeStatistics(SizeStatistics other) { | |
|
|
||
| Preconditions.checkArgument(type.equals(other.type), "Cannot merge SizeStatistics of different types"); | ||
| unencodedByteArrayDataBytes = Math.addExact(unencodedByteArrayDataBytes, other.unencodedByteArrayDataBytes); | ||
| for (int i = 0; i < repetitionLevelHistogram.size(); i++) { | ||
| repetitionLevelHistogram.set( | ||
| i, Math.addExact(repetitionLevelHistogram.get(i), other.repetitionLevelHistogram.get(i))); | ||
|
|
||
| if (other.repetitionLevelHistogram.isEmpty()) { | ||
| repetitionLevelHistogram.clear(); | ||
| } else { | ||
| Preconditions.checkArgument( | ||
| repetitionLevelHistogram.size() == other.repetitionLevelHistogram.size(), | ||
| "Cannot merge SizeStatistics with different repetition level histogram size"); | ||
| for (int i = 0; i < repetitionLevelHistogram.size(); i++) { | ||
| repetitionLevelHistogram.set( | ||
| i, Math.addExact(repetitionLevelHistogram.get(i), other.repetitionLevelHistogram.get(i))); | ||
| } | ||
| } | ||
| for (int i = 0; i < definitionLevelHistogram.size(); i++) { | ||
| definitionLevelHistogram.set( | ||
| i, | ||
| Math.addExact( | ||
| definitionLevelHistogram.get(i), | ||
| other.getDefinitionLevelHistogram().get(i))); | ||
|
|
||
| if (other.definitionLevelHistogram.isEmpty()) { | ||
| definitionLevelHistogram.clear(); | ||
| } else { | ||
| Preconditions.checkArgument( | ||
| definitionLevelHistogram.size() == other.definitionLevelHistogram.size(), | ||
| "Cannot merge SizeStatistics with different definition level histogram size"); | ||
| for (int i = 0; i < definitionLevelHistogram.size(); i++) { | ||
| definitionLevelHistogram.set( | ||
| i, Math.addExact(definitionLevelHistogram.get(i), other.definitionLevelHistogram.get(i))); | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
not sure if there is a style for preferring null vs 0 length array (or does this flow through more easily for consuption)?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
0 length array makes it easier for other methods (e.g.
copy()) to avoidNullPointerException.