[
https://issues.apache.org/jira/browse/BEAM-5967?focusedWorklogId=240801&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-240801
]
ASF GitHub Bot logged work on BEAM-5967:
----------------------------------------
Author: ASF GitHub Bot
Created on: 12/May/19 21:29
Start Date: 12/May/19 21:29
Worklog Time Spent: 10m
Work Description: reuvenlax commented on issue #8496: [BEAM-5967] Add
handling of DynamicMessage in ProtoCoder
URL: https://github.com/apache/beam/pull/8496#issuecomment-491630816
Alex, Kenn's concern is that people who make no changes to their pipeline
(other than updating the Beam version) will be unable to update.
ProtoSchemaCoder will not have this problem as people will need to make
code changes to use it.
*From: *Alex Van Boxel <[email protected]>
*Date: *Sun, May 12, 2019 at 10:11 AM
*To: *apache/beam
*Cc: *reuvenlax, Mention
*@alexvanboxel* commented on this pull request.
> ------------------------------
>
> In
>
sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java
> <https://github.com/apache/beam/pull/8496#discussion_r283145727>:
>
> > /** Private constructor. */
> private ProtoCoder(Class<T> protoMessageClass, Set<Class<?>>
extensionHostClasses) {
> this.protoMessageClass = protoMessageClass;
> this.extensionHostClasses = extensionHostClasses;
> + this.protoMessageDescriptor = null;
> + }
> +
> + private ProtoCoder(
> + Descriptors.Descriptor protoMessageDescriptor, Set<Class<?>>
extensionHostClasses) {
> + @SuppressWarnings("unchecked")
> + Class<T> protoMessageClass = (Class<T>) DynamicMessage.class;
> + this.protoMessageClass = protoMessageClass;
> + this.extensionHostClasses = extensionHostClasses;
> + this.protoMessageDescriptor = protoMessageDescriptor;
> + }
> +
> + private void writeObject(ObjectOutputStream oos) throws IOException {
>
> Well, | don't see another way todo this. I need to get the Descriptor to
> the workers. The only way I see it working is sending the FileDescriptorSet
> as a proto steam onto the Java serialized object stream.
>
> Getting everyone on ProtoSchemaProvider will have the exact same problem
> I'm afraid. I'll know in a few days when I got a working prototype.
>
> —
> You are receiving this because you were mentioned.
> Reply to this email directly, view it on GitHub
> <https://github.com/apache/beam/pull/8496#discussion_r283145727>, or mute
> the thread
>
<https://github.com/notifications/unsubscribe-auth/AFAYJVKDNB5Q7JLUP6IWKGDPVBFUBANCNFSM4HKZOJIQ>
> .
>
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 240801)
Time Spent: 2h 20m (was: 2h 10m)
> ProtoCoder doesn't support DynamicMessage
> -----------------------------------------
>
> Key: BEAM-5967
> URL: https://issues.apache.org/jira/browse/BEAM-5967
> Project: Beam
> Issue Type: Improvement
> Components: sdk-java-core
> Affects Versions: 2.8.0
> Reporter: Alex Van Boxel
> Assignee: Alex Van Boxel
> Priority: Major
> Time Spent: 2h 20m
> Remaining Estimate: 0h
>
> The ProtoCoder does make some assumptions about static messages being
> available. The DynamicMessage doesn't have some of them, mainly because the
> proto schema is defined at runtime and not at compile time.
> Does it make sense to make a special coder for DynamicMessage or build it
> into the normal ProtoCoder.
> Here is an example of the assumtion being made in the current Codec:
> {code:java}
> try {
> @SuppressWarnings("unchecked")
> T protoMessageInstance = (T)
> protoMessageClass.getMethod("getDefaultInstance").invoke(null);
> @SuppressWarnings("unchecked")
> Parser<T> tParser = (Parser<T>) protoMessageInstance.getParserForType();
> memoizedParser = tParser;
> } catch (IllegalAccessException | InvocationTargetException |
> NoSuchMethodException e) {
> throw new IllegalArgumentException(e);
> }
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)