[ 
https://issues.apache.org/jira/browse/PARQUET-1711?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17611376#comment-17611376
 ] 

ASF GitHub Bot commented on PARQUET-1711:
-----------------------------------------

emkornfield commented on code in PR #995:
URL: https://github.com/apache/parquet-mr/pull/995#discussion_r984237501


##########
parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoSchemaConverter.java:
##########
@@ -124,35 +164,61 @@ private <T> Builder<? extends Builder<?, 
GroupBuilder<T>>, GroupBuilder<T>> addR
         .named("list");
   }
 
-  private <T> GroupBuilder<GroupBuilder<T>> addRepeatedMessage(FieldDescriptor 
descriptor, GroupBuilder<T> builder) {
-    GroupBuilder<GroupBuilder<GroupBuilder<GroupBuilder<T>>>> result =
-      builder
+  private <T> GroupBuilder<GroupBuilder<T>> addRepeatedMessage(FieldDescriptor 
descriptor, GroupBuilder<T> builder, ImmutableSetMultimap<String, Integer> 
seen, int depth) {
+    GroupBuilder<GroupBuilder<GroupBuilder<GroupBuilder<T>>>> result = builder
         .group(Type.Repetition.OPTIONAL).as(listType())
         .group(Type.Repetition.REPEATED)
         .group(Type.Repetition.OPTIONAL);
 
-    convertFields(result, descriptor.getMessageType().getFields());
+    convertFields(result, descriptor.getMessageType().getFields(), seen, 
depth);
 
     return result.named("element").named("list");
   }
 
-  private <T> GroupBuilder<GroupBuilder<T>> addMessageField(FieldDescriptor 
descriptor, final GroupBuilder<T> builder) {
+  private <T> Builder<? extends Builder<?, GroupBuilder<T>>, GroupBuilder<T>> 
addMessageField(FieldDescriptor descriptor, final GroupBuilder<T> builder, 
ImmutableSetMultimap<String, Integer> seen, int depth) {
+    // Prevent recursion by terminating with optional proto bytes.
+    depth += 1;
+    String typeName = getInnerTypeName(descriptor);
+    LOG.trace("addMessageField: " + descriptor.getFullName() + " type: " + 
typeName + " depth: " + depth);
+    if (typeName != null) {
+      if (seen.get(typeName).size() > maxRecursion) {
+        return builder.primitive(BINARY, 
Type.Repetition.OPTIONAL).as((LogicalTypeAnnotation) null);
+      }
+    }
+
     if (descriptor.isMapField() && parquetSpecsCompliant) {
       // the old schema style did not include the MAP wrapper around map groups
-      return addMapField(descriptor, builder);
+      return addMapField(descriptor, builder, seen, depth);
     }
+
+    seen = ImmutableSetMultimap.<String, 
Integer>builder().putAll(seen).put(typeName, depth).build();

Review Comment:
   if this gets modified every time through this method, is immutability useful?





> [parquet-protobuf] stack overflow when work with well known json type
> ---------------------------------------------------------------------
>
>                 Key: PARQUET-1711
>                 URL: https://issues.apache.org/jira/browse/PARQUET-1711
>             Project: Parquet
>          Issue Type: Bug
>    Affects Versions: 1.10.1
>            Reporter: Lawrence He
>            Priority: Major
>
> Writing following protobuf message as parquet file is not possible: 
> {code:java}
> syntax = "proto3";
> import "google/protobuf/struct.proto";
> package test;
> option java_outer_classname = "CustomMessage";
> message TestMessage {
>     map<string, google.protobuf.ListValue> data = 1;
> } {code}
> Protobuf introduced "well known json type" such like 
> [ListValue|https://developers.google.com/protocol-buffers/docs/reference/google.protobuf#listvalue]
>  to work around json schema conversion. 
> However writing above messages traps parquet writer into an infinite loop due 
> to the "general type" support in protobuf. Current implementation will keep 
> referencing 6 possible types defined in protobuf (null, bool, number, string, 
> struct, list) and entering infinite loop when referencing "struct".
> {code:java}
> java.lang.StackOverflowErrorjava.lang.StackOverflowError at 
> java.base/java.util.Arrays$ArrayItr.<init>(Arrays.java:4418) at 
> java.base/java.util.Arrays$ArrayList.iterator(Arrays.java:4410) at 
> java.base/java.util.Collections$UnmodifiableCollection$1.<init>(Collections.java:1044)
>  at 
> java.base/java.util.Collections$UnmodifiableCollection.iterator(Collections.java:1043)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:64)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.convertFields(ProtoSchemaConverter.java:66)
>  at 
> org.apache.parquet.proto.ProtoSchemaConverter.addField(ProtoSchemaConverter.java:96)
>  {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to