-
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
[BEAM-7886] Make row coder a standard coder and implement in Python #9188
Conversation
Run Portable_Python PreCommit |
29fddd1
to
f381123
Compare
Run Python PreCommit |
Run Java PreCommit |
74804ea
to
9e730ab
Compare
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.
Did a partial review on the Java and portable side. Will review the Python side code later.
...struction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java
Outdated
Show resolved
Hide resolved
...ction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java
Show resolved
Hide resolved
...nstruction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java
Show resolved
Hide resolved
17bfdbb
to
2571d89
Compare
R: @reuvenlax would you mind taking a look at the model changes and the Java changes? I can separate out the relevant commits in their own PR if that helps. |
taking a look |
I'm reviewing the Python changes |
Thank you both! Let me know if you have any questions |
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.
Only a partial review. I need to look more closely at the files under typehints/
.
PRIMITIVE_TO_ATOMIC_TYPE.update({ | ||
# In python 3, this is a no-op because str == unicode, | ||
# but in python 2 it overrides the bytes -> BYTES mapping. | ||
str: schema_pb2.AtomicType.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.
Is there a test that does conversion to/from these types?
I suspect that in Python 2 a str such as '\xff'
will fail because it's not valid UTF-8.
In other words, str
should be invalid or converted to AtomicType.BYTES in Python 2.
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.
So the runtime types don't necessarily line up exactly with the typing representation of the schema. For example even though a schema may have an attribute with np.int*
type, we still actually produce and consume int
instances, and never use np.int*
instances at runtime.
In this case, the typing might say str
, but RowCoder uses StrUtf8Coder to produce/consume instances of past.builtins.unicode
at runtime.
I agree this could be a little confusing for users. We discussed it on the ML and @robertwb suggested this approach:
In both Python 2 and Python 3 one would use str for STRING, it would decode to
past.builtins.unicode. This seems to capture the intent better than
mapping str to BYTES in Python 2 only.)
There are tests over in row_coder_test.py, and in standard_coders.yaml/standard_coders_test.py
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.
Thinking about this some more, what about just rejecting str in Python 2 (forcing the user to say unicode). We can loosen things if this becomes too cumbersome in the future (but going the other way is backwards incompatible).
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.
Do we really need to support python2? If this is going to be a burden in general, I would rather not add support for it.
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 it'll be harder to consistently exclude support for Python 2 for all schema use.
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 that makes sense. It doesn't seem too onerous to ask people to use unicode in python 2. And that's a good point that it's a backwards compatible change if we find out otherwise. I pushed a commit that does this: ecaf73c
("aliases", typing.List[unicode]), | ||
]) | ||
|
||
coders_registry.register_coder(Person, RowCoder) |
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.
Is this required for regular use?
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.
Yes it is right now. I was hesitant to make RowCoder the default coder for any NamedTuple sub-class, and I thought this was a good way to make it opt in.
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
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/PortableSchemaCoder.java
Outdated
Show resolved
Hide resolved
Trying to think of a better name than PortableSchemaCoder, but I guess this is fine for now. |
Can we just SchemaCoder as the name? I agree that this does not matter much. Would changing this in the future will be difficult? |
This is a subclass of schema coder.
Ideally we should get rid of the current row coder (make it a utility
class) and call this RowCoder.
…On Fri, Aug 23, 2019, 1:32 PM Ahmet Altay ***@***.***> wrote:
Can we just SchemaCoder as the name? I agree that this does not matter
much. Would changing this in the future will be difficult?
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#9188?email_source=notifications&email_token=AFAYJVNELJCWWFZB27BISMLQGBCNXA5CNFSM4IHXLAS2YY3PNVWWK3TUL52HS4DFVREXG43VMVBW63LNMVXHJKTDN5WW2ZLOORPWSZGOD5BIFVQ#issuecomment-524452566>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/AFAYJVKRR4FGBIY2OBBKJWLQGBCNXANCNFSM4IHXLASQ>
.
|
Let me see if I can put together a patch that does that. I think I could just get rid of the current RowCoder and move it's logic to SchemaCoder and RowCoderGenerator. |
Run Java PreCommit |
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.
Just a couple of comments, but this is looking pretty good to me.
return new CoderTranslator<RowCoder>() { | ||
@Override | ||
public List<? extends Coder<?>> getComponents(RowCoder from) { | ||
return ImmutableList.of(); |
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.
So for the time being, we're inlining everything, rather than using components. Was there a bug tracking doing better for 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.
Yes right now there's just a fixed mapping from fieldtype to coder. There's not a bug filed for using components, I was thinking that we would just continue inlining everything. Do you think we should plan on using components instead? What does that get us?
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.
For coders, if one had a coder T one was likely to have KV<K, T> for various K, an Iterable, WindowedValue for possibly several window types, and various other permutations. Coupled with the fact that leaf coders were often huge serialized blobs made for some pretty significant savings.
Maybe this'll be less of an issue in the streaming world. I think it should not be a blocker assuming we'll be able to update this in the (short-term) future.
case FLOAT: | ||
return Float.parseFloat((String) value); | ||
case DOUBLE: | ||
return Double.parseDouble((String) value); |
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 are these 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 tried to be consistent with the YAML representations used for other coders in convertValues
. We use strings for DoubleCoder
there, presumably to avoid the possibility of running into precision errors? It looks like yaml does support floating point but explicitly states it doesn't specify a required accuracy for implementations.
@lukecwik added the parseDouble
line I linked in #8205, maybe he can clarify?
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.
YAML is JSON which supports doubles (really, as its only data type). @lukecwik I'm curious, but again this is not a blocker.
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, I created BEAM-8437 lets take this conversation over there.
|
||
# Note that if this coder's schema has *fewer* attributes than the encoded | ||
# value, we just need to ignore the additional values, which will occur | ||
# here because we only decode as many values as we have coders for. |
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 how we can safely do this, as we have to pull the extra values off the stream, right?
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.
That's a good point.. I was just trying to replicate what is implemented in java's RowCoderGenerator. It seems like this would be an issue over there as well, unless something else is consuming the unread bytes? Could there be logic to do that when there's a length-prefix?
I'd be fine with just leaving this out for now and filing a jira if we can't get a satisfying answer.
("aliases", typing.List[unicode]), | ||
]) | ||
|
||
coders_registry.register_coder(Person, RowCoder) |
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
@robertwb could you take another look? |
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, thanks.
c2d8fde
to
90f8f25
Compare
Run Python PreCommit |
3 similar comments
Run Python PreCommit |
Run Python PreCommit |
Run Python PreCommit |
Finally got CI green! @robertwb is this ok to merge now? |
Excited to see this happening. Perhaps a good time to squash the fixup commits? |
Yes, go ahead and squash into meaningful commits and merge. |
Implements the beam:coder:row:v1 standard coder (the same serialization format as Java's RowCoder). Used to encode namedtuples.
90f8f25
to
f7ce06d
Compare
Run Python PreCommit |
1 similar comment
Run Python PreCommit |
@robertwb I squashed it down to reasonable commits, so I think this is ready to merge. I think the python failures have been flakes, hopefully it will pass this time. |
I can merge this once Python passes |
🎉 Thanks everyone! |
beam:coder:row:v1
, to represent row coder as a standard coder.typing
(detailed below)standard_coders.yaml
to verify compatibility between the Java and Python implementations.Beam Schemas in Python
As noted above this PR adds basic support for Beam Schemas in Python. Currently this relies on Python's
typing
module as a native Python representation of Beam Schemas.apache_beam.typehints.schemas
includes two functions,typing_from_runner_api
andtyping_to_runner_api
which convert supported typing instances to/from portable schemas. Primitive types are mapped to numpy types (e.g.np.int32
,np.double
), Arrays are mapped totyping.List[T]
, Maps are mapped totyping.Mapping[K,V]
, and Rows are mapped totyping.NamedTuple
. Logical types are not yet supported.With the changes in this PR it's possible to use Python's row coder for simple structured data types with code like the following:
Post-Commit Tests Status (on master branch)
Pre-Commit Tests Status (on master branch)
See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.