[
https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17536799#comment-17536799
]
ASF GitHub Bot commented on PARQUET-2069:
-----------------------------------------
theosib-amazon commented on PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#issuecomment-1126305627
I won't be able to add a test any time soon. Here's why.
First take note of the two parquet files attached to
https://issues.apache.org/jira/browse/PARQUET-2069.
When I implement my own Parquet reader, the fix in this PR is able to make
the "modified.parquet" file readable by ParquetMR. So what I did was copy
org.apache.parquet.avro.TestBackwardCompatibility and modify it to read a new
parquet file that I added to the resources folder. If I make my new test
TestArrayListCompatibility point to original.parquet, it reads just fine, and
the test passes. But if I make it point to modified.parquet, then I get an
exception no matter whether this PR's fix is in or not. And the exception
thrown is not the same as the exception described in the bug report. Instead, I
get this:
org.apache.parquet.io.InvalidRecordException: Parquet/Avro schema mismatch:
Avro field 'elements' not found
This has exposed some other bug in Parquet/Avro. The thing is, since this
isn't reproducible when I use my own reader, then the only way to reproduce it
is to use tests built into ParquetMR. But due to ParquetMR's unfortunate
reliance on runtime-generated code, it's impossible to run tests from the IDE,
which makes them incredibly difficult to debug. If someone has a solution to
that problem, I'd really appreciate some help.
> Parquet file containing arrays, written by Parquet-MR, cannot be read again
> by Parquet-MR
> -----------------------------------------------------------------------------------------
>
> Key: PARQUET-2069
> URL: https://issues.apache.org/jira/browse/PARQUET-2069
> Project: Parquet
> Issue Type: Bug
> Components: parquet-avro
> Affects Versions: 1.12.0
> Environment: Windows 10
> Reporter: Devon Kozenieski
> Priority: Blocker
> Attachments: modified.parquet, original.parquet, parquet-diff.png
>
>
> In the attached files, there is one original file, and one written modified
> file that results after reading the original file and writing it back with
> Parquet-MR, with a few values modified. The schema should not be modified,
> since the schema of the input file is used as the schema to write the output
> file. However, the output file has a slightly modified schema that then
> cannot be read back the same way again with Parquet-MR, resulting in the
> exception message: java.lang.ClassCastException: optional binary element
> (STRING) is not a group
> My guess is that the issue lies in the Avro schema conversion.
> The Parquet files attached have some arrays and some nested fields.
--
This message was sent by Atlassian Jira
(v8.20.7#820007)