[ 
https://issues.apache.org/jira/browse/BEAM-5967?focusedWorklogId=304123&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-304123
 ]

ASF GitHub Bot logged work on BEAM-5967:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 30/Aug/19 09:03
            Start Date: 30/Aug/19 09:03
    Worklog Time Spent: 10m 
      Work Description: alexvanboxel commented on issue #8496: [BEAM-5967] Add 
handling of DynamicMessage in ProtoCoder
URL: https://github.com/apache/beam/pull/8496#issuecomment-526523756
 
 
   @reuvenlax @kennknowles , The ProtoDomain code from the Proto Schema support 
is now backported to this pull-request. I've created a pipeline on Dataflow 
with lots of static Protobufs and performed an upgrade for **2.14.0 -> 
2.16.0-SNAPSHOT** (build locally from this PR).
   
   What is `ProtoDomain` you will ask... well it's an **immutable wrapper** 
around the protobuf descriptor set that is not serializable. It takes over 
serialization with custom read/write object. It also provides indexes on 
everything on the set (the indexes are transient). If instantiating the 
ProtoCoder with a shared ProtoDomain object equality with the DynamicMessages 
is guaranteed.
   
   How is upgradability handled? ProtoCoder has now a fixed `serialVersionUID = 
-5043999806040629525L`, this is the UID from before the changes. The reference 
to ProtoDomain and the messageName are transient and handled by the custom 
read/write object methods. 
   
   It would be great if this still made the 2.16.0 release.
 
----------------------------------------------------------------
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: 304123)
    Time Spent: 3h 50m  (was: 3h 40m)

> 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: 3h 50m
>  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
(v8.3.2#803003)

Reply via email to