[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-11-07 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

wgtmac commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r1016186398


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +139,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {
+  log.error("Warning, Avro schema doesn't match Parquet schema, falling 
back to conversion: ", e);
+  // If the Avro schema is bad, fall back to reconstructing it from the 
Parquet schema
+  avroSchema = new 
AvroSchemaConverter(configuration).convert(parquetSchema);

Review Comment:
   Is it possible to fix the converter itself as we know it is relevant to list 
(and map type mentioned in the JIRA)? 



##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {

Review Comment:
   With this fix, the reader still throws when reading from the attached file 
in the JIRA 
(https://issues.apache.org/jira/secure/attachment/13030884/original.parquet). I 
can verify that modified.parquet is fixed with the fallback. Why is that? Does 
it mean there is any corrupted schema like original.parquet in production (not 
formally released)?



##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {

Review Comment:
   Vote +1 for adding a new config parameter. The fallback mechanism 
transparently omits extra schema information which users may depend on. 
Therefore it is good to let them get the error and try the workaround setting 
popped up from the exception message.





> 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] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-11-02 Thread Sabarishan (Jira)


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

Sabarishan commented on PARQUET-2069:
-

Is there a plan to fix this issue ?

I have encountered the same issue reading a parquet file written through spark.

Spark schema:
  optional group predicateData (LIST) \{
repeated group list {
  optional binary element (STRING);
}
  }

Avro schema:
optional group predicateData (LIST) \{
  repeated group array {
optional binary element (STRING);
  }
}

The issue is 
[AvroSchemaConverter|https://github.com/apache/parquet-mr/blob/62b774cd0f0c60cfbe540bbfa60bee15929af5d4/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java#L193]
 using "array" field naming convention which is causing schema mismatch when 
reading arrays.

> 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.10#820010)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-07-25 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r928999801


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {

Review Comment:
   That's up to you. I see this change as just a fall-back in case it bombs. 
Either it'll work, or it'll bomb again, in which case we're no worse off.





> 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.10#820010)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-07-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

shangxinli commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r928310312


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {

Review Comment:
   I don't have a good solution either. I am just afraid that if we introduce 
this, there could be some unknown side effects. Given this is a problematic 
area already(I see you commented on 
https://issues.apache.org/jira/browse/PARQUET-1681), I am not confident to 
merge it now.  
   
   Or at least, we can have a feature flag to turn on/off this fix. 
   





> 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.10#820010)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-06-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r901748898


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {
+  System.err.println("Warning, Avro schema doesn't match Parquet schema, 
falling back to conversion: " + e.toString());

Review Comment:
   Oversight on my part.





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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-06-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r901740673


##
parquet-avro/src/test/java/org/apache/parquet/avro/TestArrayListCompatibility.java:
##
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.avro;
+
+import com.google.common.io.Resources;
+import org.apache.avro.generic.GenericData;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.junit.Test;
+import java.io.IOException;
+
+public class TestArrayListCompatibility {
+
+  @Test
+  public void testListArrayCompatibility() throws IOException {
+Path testPath = new 
Path(Resources.getResource("list-array-compat.parquet").getFile());
+
+Configuration conf = new Configuration();
+ParquetReader parquetReader =
+  AvroParquetReader.builder(testPath).withConf(conf).build();
+GenericData.Record firstRecord;
+try {
+  firstRecord = (GenericData.Record) parquetReader.read();
+} catch (Exception x) {
+  x.printStackTrace();

Review Comment:
   Ok, I got rid of the extra catch. I'm not sure what kind of exceptions 
parquetReader.read() can throw, though, so we'll see if we get a compile error 
from not specifying it in the function signature. :)





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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-06-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r901733632


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {

Review Comment:
   I think the underlying problem is that some versions of ParquetMR produce 
*bad schemas*, so when we try to load those same files, parsing fails, since 
the Parquet schema implicit in the file metadata doesn't match up with the 
stored Avro schema. I'm not sure what to do about bad schemas other than to 
throw them away and try a fallback.





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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-06-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

shangxinli commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r900688194


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {

Review Comment:
   I understand the targetted issue can be solved by this retry with a 
converted schema. But I am not sure if it is safe to just ignore Avro schema in 
case of exception. @rdblue @wesm Do you have some time to have a look at this? 



##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {

Review Comment:
   I understand the target issue can be solved by this retry with a converted 
schema. But I am not sure if it is safe to just ignore Avro schema in case of 
exception. @rdblue @wesm Do you have some time to have a look at this? 





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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-06-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

shangxinli commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r900687324


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java:
##
@@ -136,10 +137,22 @@ public RecordMaterializer prepareForRead(
 
 GenericData model = getDataModel(configuration);
 String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY);
-if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
-  return newCompatMaterializer(parquetSchema, avroSchema, model);
+
+try {
+  if (compatEnabled != null && Boolean.valueOf(compatEnabled)) {
+return newCompatMaterializer(parquetSchema, avroSchema, model);
+  }
+  return new AvroRecordMaterializer(parquetSchema, avroSchema, model);
+} catch (InvalidRecordException | ClassCastException e) {
+  System.err.println("Warning, Avro schema doesn't match Parquet schema, 
falling back to conversion: " + e.toString());

Review Comment:
   Any reason we don't use Log4j? 





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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-06-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

shangxinli commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r900687175


##
parquet-avro/src/test/java/org/apache/parquet/avro/TestArrayListCompatibility.java:
##
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.parquet.avro;
+
+import com.google.common.io.Resources;
+import org.apache.avro.generic.GenericData;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.junit.Test;
+import java.io.IOException;
+
+public class TestArrayListCompatibility {
+
+  @Test
+  public void testListArrayCompatibility() throws IOException {
+Path testPath = new 
Path(Resources.getResource("list-array-compat.parquet").getFile());
+
+Configuration conf = new Configuration();
+ParquetReader parquetReader =
+  AvroParquetReader.builder(testPath).withConf(conf).build();
+GenericData.Record firstRecord;
+try {
+  firstRecord = (GenericData.Record) parquetReader.read();
+} catch (Exception x) {
+  x.printStackTrace();

Review Comment:
   I think if you don't catch, it would still print out the stack. 





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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-25 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon commented on PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#issuecomment-1138044571

   This patch of yours is cool. I can't tell you without further analysis if 
it's a universal fix, but how about you make your own PR but borrow the test 
I've included in my PR? 




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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-25 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

islamismailov commented on PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#issuecomment-1137996922

   This is a "list" AND a "map" issue, not just list. If you're using Iceberg, 
good news: just apply this PR to your iceberg branch 
https://github.com/apache/iceberg/pull/3309
   
   Link to the original issue: https://github.com/apache/iceberg/issues/2962
   
   This worked for us. If you still want to fix it in parquet you might be 
interested in this change, or something along those lines (not recommended as I 
didn't fully test this change):
   
   commit 1918276ec7f01279cb9906b9378cb8986f6ad3ea
   Author: Islam Ismailov 
   Date:   Wed May 25 19:03:33 2022 +
   
   Attempt a fix on avro-parquet conversion
   
   diff --git 
a/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java 
b/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java
   index 7d1f3cab..960aae22 100644
   -

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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-25 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

islamismailov commented on PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#issuecomment-1137620395

   I debugged this some more and it looks like some of the problem is coming 
from conversion between parquet and avro. Especially if you read parquetSchema, 
convert it to avro and set projection in avro schema format, it would get 
converted back to parquet and it will look different from the original.
   
   `System.out.println("ORIGINAL PARQUET " + fileSchema);
   Schema avroSchema = new 
AvroSchemaConverter(configuration).convert(fileSchema);
   MessageType parquetSchema = new 
AvroSchemaConverter(configuration).convert(avroSchema);
   System.out.println("RECONSTRUCTED PARQUET " + parquetSchema);`




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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon commented on PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#issuecomment-1136247681

   @islamismailov Can you provide me with a parquet file and changes to the 
test bench that reproduce this error?




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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

islamismailov commented on PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#issuecomment-1136197459

   @theosib-amazon I've hit a similar issue. Tried your fix and I see this 
error unfortunately. Can you test on Map>?
   
   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:70)
   at org.apache.parquet.cli.Main.main(Main.java:187)
   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   at java.lang.reflect.Method.invoke(Method.java:498)
   at org.apache.hadoop.util.RunJar.run(RunJar.java:221)
   at org.apache.hadoop.util.RunJar.main(RunJar.java:136)
   Caused by: java.lang.ClassCastException: required binary element (STRING) 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.(AvroRecordConverter.java:539)
   at 
org.apache.parquet.avro.AvroRecordConverter$AvroCollectionConverter.(AvroRecordConverter.java:489)
   at 
org.apache.parquet.avro.AvroRecordConverter.newConverter(AvroRecordConverter.java:293)
   at 
org.apache.parquet.avro.AvroRecordConverter.(AvroRecordConverter.java:137)
   at 
org.apache.parquet.avro.AvroRecordConverter.(AvroRecordConverter.java:91)
   at 
org.apache.parquet.avro.AvroRecordMaterializer.(AvroRecordMaterializer.java:33)
   at 
org.apache.parquet.avro.AvroReadSupport.prepareForRead(AvroReadSupport.java:142)
   at 
org.apache.parquet.hadoop.InternalParquetRecordReader.initialize(InternalParquetRecordReader.java:195)
   at 
org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:156)
   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.(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)
   




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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-16 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon commented on PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#issuecomment-1127822921

   OK, check out the code changes. I've redone this completely. Now what it 
does is try out the avro schema, and if that fails, it caches the exception and 
tries again with an avro schema that it converts from the parquet schema. This 
fixes not only 2069 but at least one other bug (whose number I can't remember).




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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-16 Thread Timothy Miller (Jira)


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

Timothy Miller commented on PARQUET-2069:
-

Well, I tried modifying prepareForRead to just reconstruct the avro schema 
always from the parquet schema, but that caused another test to fail, which is 
org.apache.parquet.avro.TestGenericLogicalTypes. So in the end what I decided 
to do was try using the avro schema, but if that throws an exception, it falls 
back to conversion and tried again.

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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-13 Thread Timothy Miller (Jira)


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

Timothy Miller commented on PARQUET-2069:
-

Yup. If I force prepareForRead() to ignore the avro schema in the metadata, 
then the modified.parquet file parses just fine, even without the code in the 
PR.

What do y'all say to us ditching entirely the attempt to salvage the avro 
schema from the metadata? That would make a bunch of reported parsing problems 
go away. It seems like the file writer is modifying the file schema in a way 
that becomes incompatible with the avro schema it attempts to save. If we want 
to go with the new format, we should stop trying to use the avro schema.

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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-13 Thread Timothy Miller (Jira)


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

Timothy Miller commented on PARQUET-2069:
-

So, where does the avro schema come from in the first place? This is in 
org.apache.parquet.avro.AvroReadSupport.prepareForRead().

This IF test passes:

 
{code:java}
if (keyValueMetaData.get(AVRO_SCHEMA_METADATA_KEY) != null){code}
 

Using a parquet metadata analysis tool I wrote, we can see where this is coming 
from:

 
{code:java}
    5: List(KeyValue key_value_metadata):
        1: Struct(KeyValue key_value_metadata):
           1: string(key) = Binary("parquet.avro.schema")
           2: string(value) = 
Binary("{"type":"record","name":"spark_schema","fields":[{"name":"destination_addresses","type":["null",{"type":"array","items":{"type":"record","name":"list","fields":[{"name":"element","type":["null","string"],"default":null}]}}],"default":null},{"name":"origin_addresses","type":["null",{"type":"array","items":{"type":"record","name":"list","namespace":"list2","fields":[{"name":"element","type":["null","string"],"default":null}]}}],"default":null},{"name":"rows","type":["null",{"type":"array","items":{"type":"record","name":"list","namespace":"list3","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"","fields":[{"name":"elements","type":["null",{"type":"array","items":{"type":"record","name":"list","namespace":"list4","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"element2","fields":[{"name":"distance","type":["null",{"type":"record","name":"distance","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"duration","type":["null",{"type":"record","name":"duration","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"status","type":["null","string"],"default":null}]}],"default":null}]}}],"default":null}]}],"default":null}]}}],"default":null},{"name":"status","type":["null","string"],"default":null}]}"){code}
The parquet schema comes from the thrift data structures at the end of the 
parquet file. The decode of that is so lengthy that I decided to put it on 
pastebin, here: [https://pastebin.com/cRGPkSwH|[https://pastebin.com/cRGPkSwH].]


I'll try to put these side-by-side later and see if I can make sense of them. 
But I'm guessing that the writer messed up and made the two schemas not 
compatible.

If I hack prepareForRead() to ignore the avro schema, everything parses just 
fine.

Right now I'm rebuilding the whole thing where the fix provided in the PR is 
backed out. I'll post another comment to let you know how that goes.

 

 

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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-13 Thread Timothy Miller (Jira)


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

Timothy Miller commented on PARQUET-2069:
-

I managed to probe this just a bit. No idea why this diverges from the behavior 
when I use the simple reader program I wrote 
(https://github.com/theosib-amazon/parquet-mr-minreader), despite using the 
exact same API calls. Specifically, I'm trying to read the modified.parquet 
that's attached to this bug report.

The exception is thrown by 
org.apache.parquet.avro.AvroRecordConverter.getAvroField(), where it's 
trying to look up an element from a parquetSchema in the avroSchema. The caller 
is org.apache.parquet.avro.AvroRecordConverter.AvroRecordConverter() (the 
constructor), where we can find a loop over parquetSchema.getFields().

The parquet schema it's trying to use is this:
{noformat}
optional group element {
  optional group elements (LIST) {
    repeated group array {
      optional group element {
        optional group distance {
          optional binary text (STRING);
          optional int64 value;
        }
        optional group duration {
          optional binary text (STRING);
          optional int64 value;
        }
        optional binary status (STRING);
      }
    }
  }
}{noformat}
The avro schema is this:
{noformat}
{"type":"record","name":"list","namespace":"list3","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"","fields":[{"name":"elements","type":["null",{"type":"array","items":{"type":"record","name":"list","namespace":"list4","fields":[{"name":"element","type":["null",{"type":"record","name":"element","namespace":"element2","fields":[{"name":"distance","type":["null",{"type":"record","name":"distance","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"duration","type":["null",{"type":"record","name":"duration","namespace":"","fields":[{"name":"text","type":["null","string"],"default":null},{"name":"value","type":["null","long"],"default":null}]}],"default":null},{"name":"status","type":["null","string"],"default":null}]}],"default":null}]}}],"default":null}]}],"default":null}],"aliases":["array"]}{noformat}
There is indeed something in the avro schema called "elements". However, 
examining the avroSchema itself that is being searched from getAvroField, the 
Schema class's fieldMap only contains entries for "rows", 
"destination_address", "origin_address", and "status". It seems like these two 
schemas have very little to do with each other and somehow the wrong kind of 
thing got passed down into this code.

 

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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-13 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-11 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon commented on PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#issuecomment-1123953964

   I'm not sure how to add a whole new test. I'll see if I can figure it out. 
Also, the best way to test this would be to include the parquet file from the 
bug report, and I'm not sure where I'd put that in the source tree.
   
   Any suggestions would be much appreciated.




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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-11 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

shangxinli commented on PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#issuecomment-1123948507

   Can you add tests? 




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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-05-11 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

shangxinli commented on code in PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#discussion_r870479919


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroRecordConverter.java:
##
@@ -866,6 +866,20 @@ static boolean isElementType(Type repeatedType, Schema 
elementSchema) {
 } else if (elementSchema != null &&
 elementSchema.getType() == Schema.Type.RECORD) {
   Schema schemaFromRepeated = 
CONVERTER.convert(repeatedType.asGroupType());
+
+  // Fix for PARQUET-2069
+  // ParquetMR breaks compatibility with itself by including a JSON 
+  // representation of a schema that names a record "list", when
+  // it should be named "array" to match with the rest of the metadata.
+  // Inserting this code allows Avro to detect that the "array" and "list"
+  // types are compatible. Since this alias is being added to something
+  // that is the result of parsing JSON, we can't add the alias at the
+  // time of construction. Therefore we have to do it here where the the 
data
+  // structures have been unwrapped to the point where we have the 
+  // incompatible structure and can add the necessary alias.
+  if (elementSchema.getName().equals("list")) 
elementSchema.addAlias("array", "");

Review Comment:
   Follow the above standard like line 866. 





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


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-04-14 Thread Timothy Miller (Jira)


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

Timothy Miller commented on PARQUET-2069:
-

I found a fix for the problem. This is going to look like an ugly hack-fix, but 
there doesn't seem to be any other place to put this, since the schema being 
unwound here isn't constructed in ParquetMR but is rather just some JSON that's 
been parsed out of the parquet file. The solution is this.

In AvroRecordConverter.AvroRecordConverter.isElementType, just before this call 
to checkReaderWriterCompatibility, I made this change:

{{+      if (elementSchema.getName().equals("list")) 
elementSchema.addAlias("array", "");}}
{{+  if (elementSchema.getName().equals("array")) 
elementSchema.addAlias("list", "");}}
{{  }}{{{}if (checkReaderWriterCompatibility(elementSchema, 
schemaFromRepeated){}}}{{{}{}}}{{{}{}}}

The following is the PR I created to fix this: 
https://github.com/apache/parquet-mr/pull/957

If this is the wrong way to do it, maybe it'll get the attention of someone who 
understands this better than I do.

> 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.1#820001)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-04-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2069:
-

theosib-amazon opened a new pull request, #957:
URL: https://github.com/apache/parquet-mr/pull/957

   This PR addresses the following JIRA entry:
   https://issues.apache.org/jira/browse/PARQUET-2069
   
   ParquetMR breaks compatibility with itself by including a JSON 
representation of a schema that names a record "list", when it should be named 
"array" to match with the rest of the metadata. The proposed change allows Avro 
to detect that the "array" and "list" types are compatible. 




> 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.1#820001)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-04-14 Thread Timothy Miller (Jira)


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

Timothy Miller commented on PARQUET-2069:
-

This appears to occur due to the reader and writer schemas having a name 
mismatch on the record I mentioned earlier. The problem appears to be either in 
Avro or in how Avro is used.

When comparing these structures for compatibility, there this method in 
org.apache.avro.SchemaCompatibility is being used:

{{  public static boolean schemaNameEquals(final Schema reader, final Schema 
writer) {}}
{{    if (objectsEqual(reader.getName(), writer.getName())) {}}
{{      return true;}}
{{    }}}
{{    // Apply reader aliases:}}
{{    return reader.getAliases().contains(writer.getFullName());}}
{{  }}}

Evidently, when the reader schema was set up, no alias was added that would 
allow "list" and "array" to match here, and this causes the failure. This 
method is called by checkSchemaNames, which is called by 
calculateCompatibility, which is called by getCompatibility, which is called by 
another getCompatibility, which is called from checkReaderWriterCompatibility, 
which is called from org.apache.parquet.avro.AvroRecordConverter.isElementType.

 

> 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.1#820001)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-04-07 Thread Timothy Miller (Jira)


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

Timothy Miller commented on PARQUET-2069:
-

With the original file, the debug message says this instead:

[main] DEBUG org.apache.avro.SchemaCompatibility - Checking compatibility of 
reader 
\{"type":"record","name":"list","fields":[{"name":"element","type":["null","string"],"default":null}]}
 with 
writer 
\{"type":"record","name":"list","fields":[{"name":"element","type":["null","string"],"default":null}]}

What I can't figure out is why the the reader record name is "list" in the 
modified case, since all the lists in the original were converted to arrays. So 
it should also be an array.

> 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.1#820001)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-04-07 Thread Timothy Miller (Jira)


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

Timothy Miller commented on PARQUET-2069:
-

Here's a log message that shows why it's failing:

{{[main] DEBUG org.apache.avro.SchemaCompatibility - Checking compatibility of 
}}
{{reader 
\{"type":"record","name":"list","fields":[{"name":"element","type":["null","string"],"default":null}]}
 with }}
{{writer 
\{"type":"record","name":"array","fields":[{"name":"element","type":["null","string"],"default":null}]}}}

When {{AvroRecordConverter.newConverter(Schema schema, Type type, GenericData 
model, Class knownClass, ParentValueContainer setter)}} encounters an ARRAY 
type, it calls {{{}AvroRecordConverter.AvroCollectionConverter{}}}, which calls 
{{{}AvroRecordConverter.isElementType{}}}, which calls 
{{{}SchemaCompatibility.checkReaderWriterCompatibility{}}}. The type that 
returns is INCOMPATIBLE, and this is because the record name for the reader is 
"list" while the record name for the writer is "array", and these are 
considered incompatible.

Either some change has to be made to the compatibility check to allow array and 
list types to be considered compatible, or the writer schema has to be computed 
differently. The latter appears to be done in 
{{{}AvroSchemaConverter.convertFields{}}}, which is called from 
{{{}AvroRecordConverter.isElementType{}}}, like this:

{{                Schema schemaFromRepeated = 
CONVERTER.convert(repeatedType.asGroupType());}}

 

> 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.1#820001)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-04-07 Thread Timothy Miller (Jira)


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

Timothy Miller commented on PARQUET-2069:
-

Based on the fact that the option is named "old" list structure, I'm going to 
guess that that's been deprecated in favor of a newer array format. That 
shouldn't necessarily cause trouble to the user, and it certainly should not 
throw an exception. Since I've already started on this, I may at least try to 
figure out how to make it not crash.

The exception is thrown as a result of code in 
{{AvroRecordConverter.newConverter(Schema schema, Type type, GenericData model, 
Class knownClass, ParentValueContainer setter).}} The schema is indicating 
that the type is {{{}RECORD{}}}, while the the {{type}} variable indicates that 
the type is a string. Gotta figure out how this inconsistency is happening.

> 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.1#820001)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-04-07 Thread Devon Kozenieski (Jira)


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

Devon Kozenieski commented on PARQUET-2069:
---

The string value was actually intentionally modified with a hashing algorithm 
as a part of my application, so that value looks correct. The problem was that 
the schema of the output file had also changed, even though I only intended to 
modify the value. As of right now this issue is not a high priority for me any 
more.

> 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.1#820001)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2022-04-07 Thread Timothy Miller (Jira)


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

Timothy Miller commented on PARQUET-2069:
-

An  initial look at this suggests that the writer is corrupting the data 
somehow. I'm using "parquet-tools dump", and there are a bunch of (I guess) 
harmless changes from "list" to "array", but there's also two examples of data 
being corrupted. See the screenshot.

 

!image-2022-04-07-11-49-38-982.png!

> 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
>
>
> 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.1#820001)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2021-10-12 Thread Sebastian Bernauer (Jira)


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

Sebastian Bernauer commented on PARQUET-2069:
-

Hi [~devonk] can you please try the following parquet setting? It resolved the 
problem for us.
{{parquet.avro.write-old-list-structure=}}{{false}}

> 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
>
>
> 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.3.4#803005)


[jira] [Commented] (PARQUET-2069) Parquet file containing arrays, written by Parquet-MR, cannot be read again by Parquet-MR

2021-09-27 Thread Sebastian Bernauer (Jira)


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

Sebastian Bernauer commented on PARQUET-2069:
-

Hi together,

 

we run into the same bug. We use Apache Hudi and have a bug ticket there 
HUDI-864

Is there any timeline on this issue?

 

Thanks a lot in advance!

Regards,

Sebastian

> 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
>
>
> 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.3.4#803005)