[
https://issues.apache.org/jira/browse/BEAM-5967?focusedWorklogId=240629&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-240629
]
ASF GitHub Bot logged work on BEAM-5967:
----------------------------------------
Author: ASF GitHub Bot
Created on: 11/May/19 17:10
Start Date: 11/May/19 17:10
Worklog Time Spent: 10m
Work Description: kennknowles commented on pull request #8496:
[BEAM-5967] Add handling of DynamicMessage in ProtoCoder
URL: https://github.com/apache/beam/pull/8496#discussion_r283104093
##########
File path:
sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java
##########
@@ -269,21 +278,61 @@ public ExtensionRegistry getExtensionRegistry() {
private transient ExtensionRegistry memoizedExtensionRegistry;
private transient Parser<T> memoizedParser;
+ // Descriptor used by DynamicMessage.
+ private transient Descriptors.Descriptor protoMessageDescriptor;
+
/** 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 {
Review comment:
Hmm, interesting. It is a problem that ProtoCoder is a CustomCoder
serialized via Java serialization rather than a StructuredCoder. Not related to
your change, just noting. This essentially makes it incompatible with pipeline
update. @reuvenlax
----------------------------------------------------------------
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: 240629)
Time Spent: 1h (was: 50m)
> 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: 1h
> 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)