-
Notifications
You must be signed in to change notification settings - Fork 1.5k
PARQUET-686: Do not return min/max for the wrong order. #367
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
Conversation
| * @return the metadata blocks in the footer | ||
| * @throws IOException if an error occurs while reading the file | ||
| */ | ||
| private static final ParquetMetadata readFooter(ParquetMetadataConverter converter, long fileLen, String filePath, SeekableInputStream f, MetadataFilter filter) throws IOException { |
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.
the new converter param seems to not be used?
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.
It is, on line 493. The name is the same as the previous version's static converter so that line didn't change at all.
|
What's the upgrade path for the future, i.e. if you want to use unsigned statistics for Binary columns? This doesn't seem to address that, just disables returning statistics for those cases. |
|
This doesn't address fixing the problem and writing correct stats because that will rely on deciding how to store the order in parquet-format and how we want to expose different orderings to users. The purpose of this patch isn't to decide those, it is to fix the correctness issue by default for non-ascii characters and to provide a way to opt in to use the current behavior. |
| private final boolean useSignedStringMinMax; | ||
|
|
||
| public ParquetMetadataConverter() { | ||
| this.useSignedStringMinMax = false; |
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.
let's not have a default here.
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.
We need a no-arg constructor because the class is public and previously had no explicit constructor. To require using the boolean constructor would break binary compatibility. Do you think we should deprecate it?
| } | ||
|
|
||
| public ParquetMetadataConverter(Configuration conf) { | ||
| this.useSignedStringMinMax = conf.getBoolean("parquet.strings.use-signed-order", false); |
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.
let's have ParquetMetadataConverter(boolean useSignedStringMinMax) instead and pull up reading the configuration with the other config parameters
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.
I'd call this setting parquet.read.use-signed-order-for-unsigned-types.enabled
see: https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetInputFormat.java
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.
I changed the property to parquet.strings.signed-min-max.enabled and moved it to ParquetInputFormat. The different name is because it only affects strings.
I initially changed the parameter to a boolean as you suggested, but when we move to a better stats representation and pass the order used to produce min/max along with stats, we won't need this any more. It will be a useless part of the public API, but we may have another situation where we need a Configuration in the future, so I think it makes more sense to go with this version.
| // NOTE: See docs in CorruptStatistics for explanation of why this check is needed | ||
| if (statistics != null && !CorruptStatistics.shouldIgnoreStatistics(createdBy, type)) { | ||
| if (statistics != null && !CorruptStatistics.shouldIgnoreStatistics(createdBy, type) && | ||
| SortOrder.SIGNED == expectedOrder) { |
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.
shouldn't we move the SortOrder check to CorruptStatistics.shouldIgnoreStatistics ?
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.
See my comment below.
|
|
||
| public org.apache.parquet.column.statistics.Statistics fromParquetStatistics( | ||
| String createdBy, Statistics statistics, PrimitiveType type) { | ||
| SortOrder expectedOrder = isSignedOrderOkay(type) ? SortOrder.SIGNED : sortOrder(type); |
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.
I'd move the checks to shouldIgnoreStatistics.
It seems there are more steps than necessary to decide whether the sort order is OK.
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.
This is a short-lived change and the order used to produce the min and max should be passed along with the stats in the next release. I don't think it makes sense to pass more type info, the actual order, and whether to override the order into the shouldIgnoreStatistics method (which is public) only to change it next release, so I'd like to keep this here to minimize the changes.
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.
ok
| * @throws IOException if an error occurs while reading the file | ||
| */ | ||
| public static final ParquetMetadata readFooter(long fileLen, String filePath, SeekableInputStream f, MetadataFilter filter) throws IOException { | ||
| return readFooter(new ParquetMetadataConverter(), fileLen, filePath, f, filter); |
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.
Since this method has not been released yet, you can change its signature.
How about introducing a ParquetReadConfig class that would have the field useSignedStringMinMax (or a better name)?
That way we don't add an unnecessary dependency on hadoop.Configuration.
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.
When this is combined with #368, I was getting the Configuration from the HadoopDataSource, which is Configurable. I like the idea of the ParquetReadConfig quite a bit to start abstracting Configuration out, but I think that's a larger task than we need to do in this commit, which is already pretty big.
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.
We should not pass hadoop configurations around. see my comments on #368.
Regarding the config I was just suggesting using it in this particular method and that particular setting. Since it has not been released yet.
Once this method is released changing its signature will require deprecating it.
But adding fields to the ParquetReadConfig object will be backwards compatible.
the parameterless ParquetMetadataConverter() should be deprecated
|
|
||
| private static final Log LOG = Log.getLog(ParquetMetadataConverter.class); | ||
|
|
||
| private final boolean useSignedStringMinMax; |
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.
how about: useSignedMinMaxForUnsignedValues
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.
I think this is more accurate. The override only works for String values (utf8, enum, json) because those are the only types where it can be safe to use the signed min and max.
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.
please add this as a javadoc comment here.
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.
Given that this controls all binaries except enum/unsigned int, can we rename this something like useSignedBinaryMinMax ?
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.
This isn't for binaries, it's for all string representations.
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.
Would be nice to add a comment on this only working for strings.
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.
I think the field name is pretty clear. What information would you like to see added in a comment?
| } | ||
| } | ||
|
|
||
| private static SortOrder defaultSortOrder(PrimitiveTypeName primitive) { |
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.
please add javadoc: "used when the type has no OriginalType annotation"
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.
Will do.
| case UINT_16: | ||
| case UINT_32: | ||
| case UINT_64: | ||
| case DECIMAL: |
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.
Decimal is unsigned?
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.
Mostly. Decimals are stored as two's complement integers so unsigned byte-wise comparison works as long as you flip the sign bit, as you would to get a correct signed integer comparison using little endian bytes. That also only works if the two byte arrays are the same length. The final solution should implement a real decimal comparison, but using unsigned now will prevent the currently incorrect min and max from being used.
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.
Your answer sounds like "it depends". Then we should make it UNKNOWN for now.
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.
Good point.
| default: // includes decimal | ||
| return false; | ||
| } | ||
| } |
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.
Why not simply:
private boolean isMinMaxUsableFor(PrimitiveType type) {
SortOrder expected = sortOrder(type);
return (expected == SIGNED) || (expected == UNSIGNED && useSignedMinMaxForUnsigned);
}
the case for getOriginalType is already treated in sortOrder(). Why adding another layer here?
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.
The other method resolves the correct sort order for a logical type. This method determines whether the incorrect sort order can be used. This will only use the signed sort order for string-based types, not for unsigned integer types.
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.
I don't see why we restrict it further since the flag is already about lifting the restriction but that's fine with me if you feel we need this.
couple more comments:
- The name of the method could be clearer. What it actually does is controlling if we're going to override the result of sortOrder().
- The way the logic is organized here is a little hard to read.
If you define the accepted String types:
private static final Set<OriginalType> acceptedStringTypes = Collections.unmodifiableSet(new HashSet(Arrays.asList(
UTF8, ENUM, JSON
)));
then it boils down to a boolean expression that I find easier to understand:
boolean overrideSortOrderToUnsigned(PrimitiveType type) {
return useSignedStringMinMax
&& (type.getPrimitiveTypeName() == BINARY)
&& (type.getOriginalType() == null || acceptedStringTypes.contains(type.getOriginalType());
}
| return SortOrder.UNKNOWN; | ||
| } | ||
|
|
||
| private static SortOrder sortOrder(PrimitiveType primitive) { |
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.
s/sortOrder/expectedSortOrder ?
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.
This should return the correct order for the type, not an expected order. I wouldn't use "expected" because the order for the file is known (it will be explicit or is signed) and the correct order is determined by the primitive type. I can see how passing this later as the expected order is strange, though. I'll update that to typeSortOrder to be more clear.
julienledem
left a comment
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.
Thanks @rdblue
Overall this looks good to me.
the logic for order validation seems a little more complex than it needs to be.
I posted comments above.
|
@julienledem, I made changes and addressed your comments. Could you have another look? Thanks! |
|
|
||
| public ParquetMetadataConverter(Configuration conf) { | ||
| this(isSignedStringMinMaxEnabled(conf)); | ||
| } |
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.
remove this constructor and just have the call to isSignedStringMinMaxEnabled where this is constructed.
|
|
||
| private final boolean useSignedStringMinMax; | ||
|
|
||
| public ParquetMetadataConverter() { |
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.
If for backward compatibility mark this as deprecated.
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.
+1
edit: still figuring out github's new review system. this was supposed to be +1 to adding a deprecated annotation to this if it's only here for compatibility.
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.
Why should this be deprecated?
It's perfectly valid to create a ParquetMetadataConverter without any configuration. It could easily be that when we restructure this into parquet-common, it's the other constructor that goes away, so I don't think we should consider it deprecated.
| */ | ||
| public static boolean isSignedStringMinMaxEnabled(Configuration conf) { | ||
| return conf.getBoolean(SIGNED_ORDER_ENABLED, SIGNED_ORDER_ENABLED_DEFAULT); | ||
| } |
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.
Hey, Sorry,
I know I am the one who asked for those methods here but as @isnotinvain mentioned on #362 we don't want to pollute the api with too many settings that are of limited use.
How about we removing these methods from there and use the string in the test like you had before?
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.
Agreed.
22b2461 to
dbfb6a9
Compare
|
@julienledem, I updated this PR for your comments other than the I think we should pass the |
|
+1 LGTM |
|
I agree, it would be great if @isnotinvain could have a look. In the interest of getting a release out, I'd like to commit this in the next day or two unless there are other comments. Thanks for reviewing, @julienledem and @andreweduffy! |
|
I will take a look. I was out of town for a bit, what did we end up deciding in the discussion in parquet-format? |
|
Basically we won't use the old min/max for <, <=, >, >= checks? But we'll still use it for == and !=? |
|
@isnotinvain, this commit will prevent signed min/max from being passed up from file metadata unless you have a property set to do so for strings. This only fixes correctness. In the next release, I think the right thing to do is to pass up the stats along with the ordering that produced them, so that we can implement == and != like you're talking about. |
|
OK, thanks for clarifying. The property that controls whether to pass these up for strings -- is it only for strings, or for all binaries? Is there a reason to treat strings and binaries separately? |
|
The property is for binary without a logical type (Impala doesn't write strings annotated with UTF8 😞), and for UTF8, JSON, and ENUMs. That excludes DECIMAL, for which the min/max are always invalid, and unsigned ints that have the same problem. |
|
|
||
| private static final Log LOG = Log.getLog(ParquetMetadataConverter.class); | ||
|
|
||
| private final boolean useSignedStringMinMax; |
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.
Given that this controls all binaries except enum/unsigned int, can we rename this something like useSignedBinaryMinMax ?
|
|
||
| private final boolean useSignedStringMinMax; | ||
|
|
||
| public ParquetMetadataConverter() { |
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.
+1
edit: still figuring out github's new review system. this was supposed to be +1 to adding a deprecated annotation to this if it's only here for compatibility.
| } | ||
|
|
||
| public ParquetMetadataConverter(Configuration conf) { | ||
| this(conf.getBoolean("parquet.strings.signed-min-max.enabled", false)); |
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.
can this key live in a public static constant? and again here, maybe use 'binary' instead of 'string' ?
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.
We removed the public constant to avoid leaking this into the public API and not being able to remove it. Multiple packages reference it, so it would need to be public. I think it's fine to use a String for now.
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.
I'm a little torn about keeping this as a string. It's easy to mess things up in that case. Would it be an option to add it as a constant with a comment / annotation indicating it is liable to change?
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.
The problem is that there is no good place to add this and we don't want to add it to the public API. The long-term fix will very likely move this option outside of the converter (returning the min/max and the sort order used during aggregation), and it isn't something we want to expose in ParquetInputFormat or ParquetReader, or else people will turn it on without understanding what it does. I agree it isn't ideal, but I think this is fine until we have a permanent solution to the problem.
| * @param type a primitive type with a logical type annotation | ||
| * @return true if signed order min/max can be used with this type | ||
| */ | ||
| private boolean overrideSortOrderToSigned(PrimitiveType type) { |
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.
this is adding a lot of complication / corner cases. Do we really need to support using signed ordering for ascii only strings? It's especially confusing that we allow this for un-annotated binary columns (we just assume they are string-ish, which doesn't seem valid).
It seems like this could be simplified by simply returning the found min/max and the ordering with which they were written and letting the filter layer decide what to do there? I think you mentioned that maybe that's the plan going forward, if it is, can we mark all these extra methods around dealing with overrides as temporary / to be deleted soon?
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.
Impala writes Strings as unannotated binary, so we should consider it string-ish if useSignedStringMinMax is set.
While passing stats and the sort order should be the approach we take, I think it will require some design and thought to make sure we don't pass invalid stats to applications. If we simply added the sort order to the current stats objects, engines like Presto would keep using the bad stats without checking it. This commit will prevent applications from getting stats and then we can take the time to fix it correctly.
| * @param primitive a primitive physical type | ||
| * @return the default sort order used when the logical type is not known | ||
| */ | ||
| private static SortOrder defaultSortOrder(PrimitiveTypeName primitive) { |
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.
can you add comments explaining what is meant by default sort order? why are non-banary types involved in this?
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.
Default sort order for a primitive type is the order we think is appropriate when the logical type isn't known. This is only used for backward-compatibility with the existing static methods to convert stats.
|
For some reason, not all of my replies are shown above and you have to go to the code tab. Something to do with the new review interface. |
piyushnarang
left a comment
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.
Changes look good for the most part. Had some comments on documentation / readability.
|
|
||
| private static final Log LOG = Log.getLog(ParquetMetadataConverter.class); | ||
|
|
||
| private final boolean useSignedStringMinMax; |
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.
Would be nice to add a comment on this only working for strings.
| } | ||
|
|
||
| public ParquetMetadataConverter(Configuration conf) { | ||
| this(conf.getBoolean("parquet.strings.signed-min-max.enabled", false)); |
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.
I'm a little torn about keeping this as a string. It's easy to mess things up in that case. Would it be an option to add it as a constant with a comment / annotation indicating it is liable to change?
| org.apache.parquet.column.statistics.Statistics stats = org.apache.parquet.column.statistics.Statistics.getStatsBasedOnType(type); | ||
| // If there was no statistics written to the footer, create an empty Statistics object and return | ||
|
|
||
| // NOTE: See docs in CorruptStatistics for explanation of why this check is needed |
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.
might also want to add a line on the sort order bit.
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.
Done.
| createdBy, statistics, type.getPrimitiveTypeName(), expectedOrder); | ||
| } | ||
|
|
||
| enum SortOrder { |
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.
add a couple of lines of javadoc to indicate what this struct is for? Also what is the convention we follow for types like enums / nested classes - should they be declared at the top of the class? Or anywhere goes? (persnally prefer the top)
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.
This is internal and will be removed (or moved into common and made public) when we add the long-term fix. That's why I'm keeping all of these classes together. That way the chance of a conflict is minimized.
| UNKNOWN | ||
| } | ||
|
|
||
| private static final Set<OriginalType> STRING_TYPES = Collections |
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.
move to top?
Min and max are currently calculated using the default Java ordering that uses signed comparison for all values. This is not correct for binary types like strings and decimals or for unsigned numeric types. This commit prevents statistics accumulated using the signed ordering from being returned by ParquetMetadataConverter when the type should use the unsigned ordering. Because many binary strings are not affected by using the wrong ordering, this adds a property, parquet.strings.use-signed-order to allow overriding this change.
dbfb6a9 to
301bd3a
Compare
rdblue
left a comment
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.
@piyushnarang: I added some additional comments and javadoc and rebased.
I also replied to a couple of your comments. I don't think there's a good place to put the override property and since it's mainly for people paying attention to this thread, I think it's fine as-is. Let me know if you disagree and have a good idea about an appropriate place to put it. Thanks for reviewing!
|
|
||
| private static final Log LOG = Log.getLog(ParquetMetadataConverter.class); | ||
|
|
||
| private final boolean useSignedStringMinMax; |
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.
I think the field name is pretty clear. What information would you like to see added in a comment?
| } | ||
|
|
||
| public ParquetMetadataConverter(Configuration conf) { | ||
| this(conf.getBoolean("parquet.strings.signed-min-max.enabled", false)); |
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.
The problem is that there is no good place to add this and we don't want to add it to the public API. The long-term fix will very likely move this option outside of the converter (returning the min/max and the sort order used during aggregation), and it isn't something we want to expose in ParquetInputFormat or ParquetReader, or else people will turn it on without understanding what it does. I agree it isn't ideal, but I think this is fine until we have a permanent solution to the problem.
| org.apache.parquet.column.statistics.Statistics stats = org.apache.parquet.column.statistics.Statistics.getStatsBasedOnType(type); | ||
| // If there was no statistics written to the footer, create an empty Statistics object and return | ||
|
|
||
| // NOTE: See docs in CorruptStatistics for explanation of why this check is needed |
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.
Done.
| createdBy, statistics, type.getPrimitiveTypeName(), expectedOrder); | ||
| } | ||
|
|
||
| enum SortOrder { |
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.
This is internal and will be removed (or moved into common and made public) when we add the long-term fix. That's why I'm keeping all of these classes together. That way the chance of a conflict is minimized.
|
@rdblue thanks for the updates. 👍 |
This is used to avoiding extra public methods in ParquetFileReader.
3b8d936 to
f9d459f
Compare
|
+1 |
Min and max are currently calculated using the default Java ordering that uses signed comparison for all values. This is not correct for binary types like strings and decimals or for unsigned numeric types. This commit prevents statistics accumulated using the signed ordering from being returned by ParquetMetadataConverter when the type should use the unsigned ordering. Because many binary strings are not affected by using the wrong ordering, this adds a property, parquet.strings.use-signed-order to allow overriding this change. Author: Ryan Blue <[email protected]> Closes apache#367 from rdblue/PARQUET-686-suppress-signed-stats and squashes the following commits: f9d459f [Ryan Blue] PARQUET-686: Add getConfiguration to HadoopInputFile. 301bd3a [Ryan Blue] PARQUET-686: Address review comments. c099c35 [Ryan Blue] PARQUET-686: Do not return min/max for the wrong order.
Min and max are currently calculated using the default Java ordering that uses signed comparison for all values. This is not correct for binary types like strings and decimals or for unsigned numeric types. This commit prevents statistics accumulated using the signed ordering from being returned by ParquetMetadataConverter when the type should use the unsigned ordering. Because many binary strings are not affected by using the wrong ordering, this adds a property, parquet.strings.use-signed-order to allow overriding this change. Author: Ryan Blue <[email protected]> Closes apache#367 from rdblue/PARQUET-686-suppress-signed-stats and squashes the following commits: f9d459f [Ryan Blue] PARQUET-686: Add getConfiguration to HadoopInputFile. 301bd3a [Ryan Blue] PARQUET-686: Address review comments. c099c35 [Ryan Blue] PARQUET-686: Do not return min/max for the wrong order.
Min and max are currently calculated using the default Java ordering that uses signed comparison for all values. This is not correct for binary types like strings and decimals or for unsigned numeric types. This commit prevents statistics accumulated using the signed ordering from being returned by ParquetMetadataConverter when the type should use the unsigned ordering. Because many binary strings are not affected by using the wrong ordering, this adds a property, parquet.strings.use-signed-order to allow overriding this change. Author: Ryan Blue <[email protected]> Closes apache#367 from rdblue/PARQUET-686-suppress-signed-stats and squashes the following commits: f9d459f [Ryan Blue] PARQUET-686: Add getConfiguration to HadoopInputFile. 301bd3a [Ryan Blue] PARQUET-686: Address review comments. c099c35 [Ryan Blue] PARQUET-686: Do not return min/max for the wrong order.
Min and max are currently calculated using the default Java ordering
that uses signed comparison for all values. This is not correct for
binary types like strings and decimals or for unsigned numeric types.
This commit prevents statistics accumulated using the signed ordering
from being returned by ParquetMetadataConverter when the type should use
the unsigned ordering.
Because many binary strings are not affected by using the wrong
ordering, this adds a property, parquet.strings.use-signed-order to
allow overriding this change.