[ 
https://issues.apache.org/jira/browse/PARQUET-2140?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vinoo Ganesh updated PARQUET-2140:
----------------------------------
    Description: 
I am finding that parquet-cli throws when trying to read UUID values. 

Attached to this bug report is a parquet file with 2 columns, message encoded 
as byte-array and number encoded as fixed length byte array (UUID). This file 
was written by my .net implementation of parquet specification. The file has 
one row worth of data and is readable by parquet-cpp.

+Schema as read by parquet-cli:+

message root

{   required binary Message (STRING);   required fixed_len_byte_array(16) 
Number (UUID); }

+Values as read by parquet-cpp:+

— Values —
Message                       |Number                        |
First record                  |215 48 212 219 218 57 169 67 166 116 7 79 44 227 
50 17 |

 

+Here is the exception stack from parquet-cli when trying to read uuid values:+
{code:java}
Caused by: org.apache.parquet.io.ParquetDecodingException: The requested schema 
is not compatible with the file schema. incompatible types: required binary 
Number (STRING) != required fixed_len_byte_array(16) Number (UUID)
        at 
org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101)
        at 
org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93)
        at 
org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602)
        at 
org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83)
        at 
org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57)
        at org.apache.parquet.schema.MessageType.accept(MessageType.java:55)
        at 
org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162)
        at 
org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135)
        at 
org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225)
 {code}
 I debugged parquet-cli code and found that parquet-cli is trying to project 
the UUID as a string and later on that throws as these types are not 
compatible? 

 

+Source code references:+

At AvroReadSupport.java, line 97

~~~~~~~~~~~~

    String requestedProjectionString = 
configuration.get(AVRO_REQUESTED_PROJECTION);

    if (requestedProjectionString != null)

{       Schema avroRequestedProjection = new 
Schema.Parser().parse(requestedProjectionString);       projection = new 
AvroSchemaConverter(configuration).convert(avroRequestedProjection);     }

~~~~~~~~~~~~

 

+Debugger values for+

requestedProjectionString=

{"type":"record","name":"root","fields":[\\{"name":"Message","type":"string"}

,\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]}

[Note that `Number` now has a type of `string` and a logicalType of `uuid`]

 

At ColumnIOFactory.java line 93

~~~~~~~~~~~~

incompatibleSchema(primitiveType, currentRequestedType);

~~~~~~~~~~~~

+Debugger values for+ 

primitiveType = required fixed_len_byte_array(16) Number (UUID)

currentRequestedType = required binary Number (STRING)

 

and this will throw.

 

If I skip over the projection code in AvroReadSupport, parquet-cli is able to 
read my file.

I am not sure if the bug is in parquet-cli or parquet-mr or in the library I 
used to encode this file. The fact that parquet-cpp is able to read it gives me 
some confidence to say that the problem is either in parquet-cli or parquet-mr.

Please point me in the right direction if I could verify this UUID 
roundtripping purely from parquet-mr itself in form of an unit-test. Happy to 
contribute tests or fix if needed.

  was:
I am finding that parquet-cli throws when trying to read UUID values. 

Attached to this bug report is a parquet file with 2 columns, message encoded 
as byte-array and number encoded as fixed length byte array (UUID). This file 
was written by my .net implementation of parquet specification. The file has 
one row worth of data and is readable by parquet-cpp.

+Schema as read by parquet-cli:+

message root {

  required binary Message (STRING);

  required fixed_len_byte_array(16) Number (UUID);

}

+Values as read by parquet-cpp:+

--- Values ---
Message                       |Number                        |
First record                  |215 48 212 219 218 57 169 67 166 116 7 79 44 227 
50 17 |

 

+Here is the exception stack from parquet-cli when trying to read uuid values:+

Caused by: org.apache.parquet.io.ParquetDecodingException: The requested schema 
is not compatible with the file schema. incompatible types: required binary 
Number (STRING) != required fixed_len_byte_array(16) Number (UUID)

        at 
org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101)

        at 
org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93)

        at 
org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602)

        at 
org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83)

        at 
org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57)

        at org.apache.parquet.schema.MessageType.accept(MessageType.java:55)

        at 
org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162)

        at 
org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135)

        at 
org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225)

 

 I debugged parquet-cli code and found that parquet-cli is trying to project 
the UUID as a string and later on that throws as these types are not 
compatible? 

 

+Source code references:+

At AvroReadSupport.java, line 97

~~~~~~~~~~~~

    String requestedProjectionString = 
configuration.get(AVRO_REQUESTED_PROJECTION);

    if (requestedProjectionString != null) {

      Schema avroRequestedProjection = new 
Schema.Parser().parse(requestedProjectionString);

      projection = new 
AvroSchemaConverter(configuration).convert(avroRequestedProjection);

    }

~~~~~~~~~~~~

 

+Debugger values for+ 

requestedProjectionString=

{"type":"record","name":"root","fields":[\{"name":"Message","type":"string"},\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]}

[Note that `Number` now has a type of `string` and a logicalType of `uuid`]

 

At ColumnIOFactory.java line 93

~~~~~~~~~~~~

incompatibleSchema(primitiveType, currentRequestedType);

~~~~~~~~~~~~

+Debugger values for+ 

primitiveType = required fixed_len_byte_array(16) Number (UUID)

currentRequestedType = required binary Number (STRING)

 

and this will throw.

 

If I skip over the projection code in AvroReadSupport, parquet-cli is able to 
read my file.

I am not sure if the bug is in parquet-cli or parquet-mr or in the library I 
used to encode this file. The fact that parquet-cpp is able to read it gives me 
some confidence to say that the problem is either in parquet-cli or parquet-mr.

Please point me in the right direction if I could verify this UUID 
roundtripping purely from parquet-mr itself in form of an unit-test. Happy to 
contribute tests or fix if needed.


> parquet-cli unable to read UUID values
> --------------------------------------
>
>                 Key: PARQUET-2140
>                 URL: https://issues.apache.org/jira/browse/PARQUET-2140
>             Project: Parquet
>          Issue Type: Bug
>          Components: parquet-cli
>            Reporter: Balaji K
>            Priority: Minor
>         Attachments: guid.parquet
>
>
> I am finding that parquet-cli throws when trying to read UUID values. 
> Attached to this bug report is a parquet file with 2 columns, message encoded 
> as byte-array and number encoded as fixed length byte array (UUID). This file 
> was written by my .net implementation of parquet specification. The file has 
> one row worth of data and is readable by parquet-cpp.
> +Schema as read by parquet-cli:+
> message root
> {   required binary Message (STRING);   required fixed_len_byte_array(16) 
> Number (UUID); }
> +Values as read by parquet-cpp:+
> — Values —
> Message                       |Number                        |
> First record                  |215 48 212 219 218 57 169 67 166 116 7 79 44 
> 227 50 17 |
>  
> +Here is the exception stack from parquet-cli when trying to read uuid 
> values:+
> {code:java}
> Caused by: org.apache.parquet.io.ParquetDecodingException: The requested 
> schema is not compatible with the file schema. incompatible types: required 
> binary Number (STRING) != required fixed_len_byte_array(16) Number (UUID)
>         at 
> org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.incompatibleSchema(ColumnIOFactory.java:101)
>         at 
> org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:93)
>         at 
> org.apache.parquet.schema.PrimitiveType.accept(PrimitiveType.java:602)
>         at 
> org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visitChildren(ColumnIOFactory.java:83)
>         at 
> org.apache.parquet.io.ColumnIOFactory$ColumnIOCreatorVisitor.visit(ColumnIOFactory.java:57)
>         at org.apache.parquet.schema.MessageType.accept(MessageType.java:55)
>         at 
> org.apache.parquet.io.ColumnIOFactory.getColumnIO(ColumnIOFactory.java:162)
>         at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:135)
>         at 
> org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:225)
>  {code}
>  I debugged parquet-cli code and found that parquet-cli is trying to project 
> the UUID as a string and later on that throws as these types are not 
> compatible? 
>  
> +Source code references:+
> At AvroReadSupport.java, line 97
> ~~~~~~~~~~~~
>     String requestedProjectionString = 
> configuration.get(AVRO_REQUESTED_PROJECTION);
>     if (requestedProjectionString != null)
> {       Schema avroRequestedProjection = new 
> Schema.Parser().parse(requestedProjectionString);       projection = new 
> AvroSchemaConverter(configuration).convert(avroRequestedProjection);     }
> ~~~~~~~~~~~~
>  
> +Debugger values for+
> requestedProjectionString=
> {"type":"record","name":"root","fields":[\\{"name":"Message","type":"string"}
> ,\{"name":"Number","type":{"type":"string","logicalType":"uuid"}}]}
> [Note that `Number` now has a type of `string` and a logicalType of `uuid`]
>  
> At ColumnIOFactory.java line 93
> ~~~~~~~~~~~~
> incompatibleSchema(primitiveType, currentRequestedType);
> ~~~~~~~~~~~~
> +Debugger values for+ 
> primitiveType = required fixed_len_byte_array(16) Number (UUID)
> currentRequestedType = required binary Number (STRING)
>  
> and this will throw.
>  
> If I skip over the projection code in AvroReadSupport, parquet-cli is able to 
> read my file.
> I am not sure if the bug is in parquet-cli or parquet-mr or in the library I 
> used to encode this file. The fact that parquet-cpp is able to read it gives 
> me some confidence to say that the problem is either in parquet-cli or 
> parquet-mr.
> Please point me in the right direction if I could verify this UUID 
> roundtripping purely from parquet-mr itself in form of an unit-test. Happy to 
> contribute tests or fix if needed.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to