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

J Y commented on PARQUET-2181:
------------------------------

i've attached some parquet files that all read fine using parquet-tools (both 
the deprecated version from parquet-mr and the [one written in 
python|https://github.com/ktrueda/parquet-tools]) *but do not read at all using 
parquet-cli*.  parquet-cli's meta command works fine.

it turns out, there's other stack traces when trying to use parquet-cli to read 
these files.  in addition to the repeated primitive issue highlighted 
originally, there's 2 other issues like the following exhibited in these files:

{quote}--- 
./raw/delivery-log/dt=2022-08-10/hour=04/part-02a95a0e-bd21-4476-9d0f-d1896687b12a-0
                                                                         
Argument error: Map key type must be binary (UTF8): required int32 key          
                                                                                
 

--- 
./raw/user/dt=2022-08-10/hour=04/part-8cac1d0c-fb7f-4a9a-b77e-b3dd59f89333-0    
                                                                             
Unknown error                                                                   
                                                                                
 
java.lang.RuntimeException: Failed on record 0                                  
                                                                                
 
        at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:86)   
                                                                                
 
        at org.apache.parquet.cli.Main.run(Main.java:157)                       
                                                                                
 
        at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)            
                                                                                
 
        at org.apache.parquet.cli.Main.main(Main.java:187)                      
                                                                                
 
Caused by: org.apache.parquet.io.InvalidRecordException: Parquet/Avro schema 
mismatch: Avro field 'null_value' not found
        at 
org.apache.parquet.avro.AvroRecordConverter.getAvroField(AvroRecordConverter.java:221)
       {quote}                                                         

is using AvroReadSupport and AvroRecrodConverter the right way to go for 
protobufs?  it looks like the parquet-tools that was deprecated in 1.12.3+ 
doesn't use the parquet-avro approach to reading (it uses [its own 
SimpleReadSupport 
approach|https://github.com/apache/parquet-mr/tree/apache-parquet-1.12.2/parquet-tools-deprecated/src/main/java/org/apache/parquet/tools/read]),
 which makes sense to me given the underlying schema and data written in 
parquet-protobuf generated files are not avro...

should we move parquet-cli back to SimpleReadSupport instead of relying on what 
appears to be a broken AvroReadSupport when dealing with proto generated files?

> parquet-cli fails at supporting parquet-protobuf generated schemas that have 
> repeated primitives in them
> --------------------------------------------------------------------------------------------------------
>
>                 Key: PARQUET-2181
>                 URL: https://issues.apache.org/jira/browse/PARQUET-2181
>             Project: Parquet
>          Issue Type: Bug
>          Components: parquet-cli
>            Reporter: J Y
>            Priority: Major
>         Attachments: samples.tgz
>
>
> i generated a parquet file using a protobuf with this proto definition:
> {code:java}
> message IndexPath {
>   // Index of item in path.
>   repeated int32 index = 1;
> }
> message SomeEvent {
>   // truncated/obfuscated wrapper
>   optional IndexPath client_position = 1;
> }
> {code}
> this gets translated to the following parquet schema using the new compliant 
> schema for lists:
> {code:java}
> message SomeEvent {
>   optional group client_position = 1 {
>     optional group index (LIST) = 1 {
>       repeated group list {
>         required int32 element;
>       }
>     }
>   }
> }
> {code}
> this causes parquet-cli cat to barf on a file containing these events:
> {quote}java.lang.RuntimeException: Failed on record 0
>         at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:86)
>         at org.apache.parquet.cli.Main.run(Main.java:157)
>         at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
>         at org.apache.parquet.cli.Main.main(Main.java:187)
> Caused by: java.lang.ClassCastException: required int32 element is not a group
>         at org.apache.parquet.schema.Type.asGroupType(Type.java:248)
>         at 
> org.apache.parquet.avro.AvroRecordConverter.newConverter(AvroRecordConverter.java:284)
>         at 
> org.apache.parquet.avro.AvroRecordConverter.newConverter(AvroRecordConverter.java:228)
>         at 
> org.apache.parquet.avro.AvroRecordConverter.access$100(AvroRecordConverter.java:74)
>         at 
> org.apache.parquet.avro.AvroRecordConverter$AvroCollectionConverter$ElementConverter.<init>(AvroRecordConverter.java:539)
>         at 
> org.apache.parquet.avro.AvroRecordConverter$AvroCollectionConverter.<init>(AvroRecordConverter.java:489)
>         at 
> org.apache.parquet.avro.AvroRecordConverter.newConverter(AvroRecordConverter.java:293)
>         at 
> org.apache.parquet.avro.AvroRecordConverter.<init>(AvroRecordConverter.java:137)
>         at 
> org.apache.parquet.avro.AvroRecordConverter.newConverter(AvroRecordConverter.java:284)
>         at 
> org.apache.parquet.avro.AvroRecordConverter.<init>(AvroRecordConverter.java:137)
>         at 
> org.apache.parquet.avro.AvroRecordConverter.<init>(AvroRecordConverter.java:91)
>         at 
> org.apache.parquet.avro.AvroRecordMaterializer.<init>(AvroRecordMaterializer.java:33)
>         at 
> org.apache.parquet.avro.AvroReadSupport.prepareForRead(AvroReadSupport.java:142)
>         at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.initialize(InternalParquetRecordReader.java:190)
>         at 
> org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:166)
>         at 
> org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
>         at 
> org.apache.parquet.cli.BaseCommand$1$1.advance(BaseCommand.java:363)
>         at org.apache.parquet.cli.BaseCommand$1$1.<init>(BaseCommand.java:344)
>         at org.apache.parquet.cli.BaseCommand$1.iterator(BaseCommand.java:342)
>         at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:73)
>         ... 3 more
> {quote}
> using the old parquet-tools binary to cat this file works fine.



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

Reply via email to