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

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

jinyius commented on PR #988:
URL: https://github.com/apache/parquet-mr/pull/988#issuecomment-1232470935

   hmm... what timing.  i actually have a pr for what i think is a more robust 
approach that truncates at an arbitrary recursion depth by putting the 
remaining recursion levels into a binary blob.  this approach lets downstream 
querying things query the non-truncated parts fine, and allows for udfs to be 
defined to reinstantiate the truncated recursed fields.
   
   i didn't submit the pr for merge quite yet b/c i'm busy trying to finish off 
the overall project i needed this for at work, so it's just coded against 
1.12.3 and not head.
   
   ptal, and if everyone likes my proposal, i can spend a few cycles and move 
it to head:
   
   schema converter pr:
    - https://github.com/promotedai/parquet-mr/pull/1
   write support pr:
   -  https://github.com/promotedai/parquet-mr/pull/2




> [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