-
Notifications
You must be signed in to change notification settings - Fork 4.2k
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
Store logical type values in Row instead of base values #11074
Conversation
R: @kanterov |
I can review the PR only this evening, but the first question that pops into my mind is:
|
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.
Overall, the idea looks good. We have room to keep using base values in some implementation of Row, that would have specialized implementation of Coder. getValue
being "cheap" is that what I would expect from such API as a user, that what makes it better than the existing approach.
I've left a few comments, didn't have time to go through some pieces (e.g. ByteBuddy changes).
.put(TypeName.BOOLEAN, BooleanCoder.of()) | ||
.build(); | ||
|
||
private static class LogicalTypeCoder extends Coder { |
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.
There is similar code in cb3dce0#diff-7ed3da1c0f656d0645423b9dab4da881R31
There are a few differences that might make sense to incorporate:
- don't use rawtypes for better typesafety
- override
structuralValue
- explicitly override
constentWithEquals
- I wish we could do something better for
contentWithEquals
, but out of my head it would be unexpected to add any of this information toLogicalType
- having
registerByteSizeObserver
,isRegisterByteSizeObserverCheap
might make sense getCoderArguments
could returnbaseCoder
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.
Made these changes.
We should probably update SchemaCoder as well to do this properly, but that can be in a different PR IMO.
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoderHelpers.java
Show resolved
Hide resolved
sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/GroupTest.java
Show resolved
Hide resolved
sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/GroupTest.java
Outdated
Show resolved
Hide resolved
sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java
Outdated
Show resolved
Hide resolved
sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java
Outdated
Show resolved
Hide resolved
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java
Outdated
Show resolved
Hide resolved
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java
Show resolved
Hide resolved
sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoderGenerator.java
Outdated
Show resolved
Hide resolved
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FromRowUsingCreator.java
Show resolved
Hide resolved
@alexvanboxel Yes, that is exactly true. This means that the logical type value must now implement equals and hashCode, whereas before it didn't (because we stored only the base value). We could work around this by always converting to the base type inside of equals and hashCode, though that might be a bit expensive. However value types without a proper equals are generally discouraged in Java, so I think we can simply document that equals is required. |
Documenting the requirement seems reasonable. |
Run Java PreCommit |
@alexvanboxel @kanterov any more comments? @alexvanboxel this will conflict badly with your PR I think (as you moved code into a new file, which makes merges tricky) so we need to be quite careful with how we merge these PRs. |
No more comment, I like the consistency. We're using some logical types into our pipelines, so I will make a custom build before the release to see if all goes well. I've merged my PR onto master as soon as I saw the LGTM, so I think the easiest thing is to rebase this branch onto master and handle the schema conflicts. I'll review as soon as this is done. |
40b9b65
to
71ea5fc
Compare
@alexvanboxel I've rebased on top of your PR now. the only conflicts were in Row.java, which I've now handled. |
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.
LGTM except a small comment, but we need to realize this is a braking change for pipelines that use logical types. Probably some IO's that are schema aware will brake as well.
.apply(Create.of(row)) | ||
.setRowSchema(outputRowSchema) | ||
.apply(Create.of(row).withRowSchema(inputRowSchema)) | ||
// .setRowSchema(outputRowSchema) |
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 comment
71ea5fc
to
4d27173
Compare
Run SQL Postcommit |
4d27173
to
6e77596
Compare
Run SQL Postcommit |
Run Java PreCommit |
1 similar comment
Run Java PreCommit |
public void encode(InputT value, OutputStream outStream) throws CoderException, IOException { | ||
BaseT baseType = logicalType.toBaseType(value); | ||
if (isDateTime) { | ||
baseType = (BaseT) ((ReadableInstant) baseType).toInstant(); |
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 is this check 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.
It's to maintain the current invariant that any ReadableInstant can be passed in, while the current InstantCoder requires an Instant. This used to be enforced in the Row builder because we would call toBaseType there.
We need to redo DateTime types I think (really we have a timestamp type, not a datetime type), but until then I wanted to maintain the existing behavior.
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. Makes sense. Agree that DateTime types need to be redone.
Run Java PreCommit |
4 similar comments
Run Java PreCommit |
Run Java PreCommit |
Run Java PreCommit |
Run Java PreCommit |
Run Java PreCommit |
After 8 runs, the only Java Precommit failures have been random flakes (e.g. in Flink tests). |
return (T) oneOfType.createValue(oneOfValue.getCaseType(), convertedOneOfField); | ||
} else if (type.getTypeName().isLogicalType()) { | ||
// Getters are assumed to return the base type. | ||
return (T) type.getLogicalType().toInputType(fieldValue); |
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.
@reuvenlax Wondering about handling of logical types here: I don't think it matters too much as I couldn't find any usage of logical types in the various GetterBasedSchemaProvider
s. But I ran into this when testing various approaches.
When using a logical type with RowWithGetters
, getterTarget
will contain a corresponding field of input type. The getter will have to convert that field value to base type to match this assumption, just to convert it back to input type here. And vice versa for the setter here.
Wouldn't it be more performant to expect getters to return the input 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.
Alternatively, I wonder if we could we add a way to register getters that are assumed to return the input 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.
There was a past world where the getters supported returning either logical type or base type. This was suppose to be whatever the previous transform provided (or the base type if coming from a coder). The goal was to save a conversion from base type to logical type for values that were just passed along. I believe the only use case was SQL and I removed that in #13930 because it is broken and like you notice here more expensive.
It should be possible to do more simplification such that we always work with logical types instead of base types. Eventually we will want a pass-through optimization again, but it will need to be at a lower level than logical types. (The only expensive type today is String, which isn't a logical type.)
@kanterov @alexvanboxel I believe this PR will fix the issues you've both had with logical types.
After some thought, I think we're better off storing the logical type value in the Row object. It potentially makes SchemaCoder a tiny bit slower, but unlikely this is noticeable. One bonus: it means that storage for OneOf types is far more memory efficient, as we no longer need to store the entire row.