[ https://issues.apache.org/jira/browse/BEAM-5967?focusedWorklogId=240900&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-240900 ]
ASF GitHub Bot logged work on BEAM-5967: ---------------------------------------- Author: ASF GitHub Bot Created on: 13/May/19 06:34 Start Date: 13/May/19 06:34 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-491693602 @reuvenlax @kennknowles if upgradeability is a problem I think I handled this correctly, please note this extract from the serialization spec: > 5.6.2 Compatible Changes > Adding writeObject/readObject methods - If the version reading the stream has these methods then readObject is expected, as usual, to read the required data written to the stream by the default serialization. It should call defaultReadObject first before reading any optional data. The writeObject method is expected as usual to call defaultWriteObject to write the required data and then may write optional data. ref: https://docs.oracle.com/javase/8/docs/platform/serialization/spec/version.html This is why: 1. on line 285 the descriptor is added `transient` 2. on line 303 the `writeObject` (and `readObject`) as added as in the spec (compatible) 3. on line 304 the `oos.defaultWriteObject();` is added as in the spec (compatible) 4. on line 305 writing the `Descriptor` **only** kicks in when the Class is `DynamicMessage`. Note that we will have no one that will have a pipeline running with `DynamicMessage`. This is because `DynamicMessage` doesn't have a `getDescriptor()` without arguments. ---------------------------------------------------------------- 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 240900) Time Spent: 2.5h (was: 2h 20m) > 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: 2.5h > 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)