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

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

                Author: ASF GitHub Bot
            Created on: 10/Oct/19 16:54
            Start Date: 10/Oct/19 16:54
    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_r333628729
 
 

 ##########
 File path: 
sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java
 ##########
 @@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.protobuf;
+
+import com.google.protobuf.DescriptorProtos;
+import com.google.protobuf.Descriptors;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * ProtoDomain is a container class for Protobuf descriptors. By using a 
domain for all descriptors
+ * that are related to each other the FileDescriptorSet needs to be serialized 
only once in the
+ * graph.
+ *
+ * <p>Using a domain also grantees that all Descriptors have object equality, 
just like statically
+ * compiled Proto classes Descriptors. A lot of Java code isn't used to the 
new DynamicMessages an
+ * assume always Object equality. Because of this the domain class is 
immutable.
+ *
+ * <p>ProtoDomains aren't assumed to be used on with normal Message 
descriptors, only with
+ * DynamicMessage descriptors.
+ */
+public final class ProtoDomain implements Serializable {
+  public static final long serialVersionUID = 1L;
+  private transient DescriptorProtos.FileDescriptorSet fileDescriptorSet;
+  private transient int hashCode;
+
+  private transient Map<String, Descriptors.FileDescriptor> fileDescriptorMap;
+  private transient Map<String, Descriptors.Descriptor> descriptorMap;
+
+  private transient Map<Integer, Descriptors.FieldDescriptor> fileOptionMap;
+  private transient Map<Integer, Descriptors.FieldDescriptor> messageOptionMap;
+  private transient Map<Integer, Descriptors.FieldDescriptor> fieldOptionMap;
+
+  ProtoDomain() {
+    this(DescriptorProtos.FileDescriptorSet.newBuilder().build());
+  }
+
+  private ProtoDomain(DescriptorProtos.FileDescriptorSet fileDescriptorSet) {
+    this.fileDescriptorSet = fileDescriptorSet;
+    hashCode = java.util.Arrays.hashCode(this.fileDescriptorSet.toByteArray());
+    crosswire();
+  }
+
+  private static Map<String, DescriptorProtos.FileDescriptorProto> 
extractProtoMap(
+      DescriptorProtos.FileDescriptorSet fileDescriptorSet) {
+    HashMap<String, DescriptorProtos.FileDescriptorProto> map = new 
HashMap<>();
+    fileDescriptorSet.getFileList().forEach(fdp -> map.put(fdp.getName(), 
fdp));
+    return map;
+  }
+
+  private static Descriptors.FileDescriptor convertToFileDescriptorMap(
 
 Review comment:
   `@Nullable`. I'm a bit sad our analyses didn't catch this.
 
----------------------------------------------------------------
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: 326449)
    Time Spent: 4h 10m  (was: 4h)

> 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
>             Fix For: 2.17.0
>
>          Time Spent: 4h 10m
>  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.4#803005)

Reply via email to