You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by GitBox <gi...@apache.org> on 2022/04/14 16:09:23 UTC

[GitHub] [parquet-mr] theosib-amazon opened a new pull request, #957: PARQUET-2069: Allow list and array record types to be compatible.

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. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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<T> 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<T>(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? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] islamismailov commented on pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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<string, struct<bigint, string>>?
   
   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.<init>(AvroRecordConverter.java:539)
           at org.apache.parquet.avro.AvroRecordConverter$AvroCollectionConverter.<init>(AvroRecordConverter.java:489)
           at org.apache.parquet.avro.AvroRecordConverter.newConverter(AvroRecordConverter.java:293)
           at org.apache.parquet.avro.AvroRecordConverter.<init>(AvroRecordConverter.java:137)
           at org.apache.parquet.avro.AvroRecordConverter.<init>(AvroRecordConverter.java:91)
           at org.apache.parquet.avro.AvroRecordMaterializer.<init>(AvroRecordMaterializer.java:33)
           at org.apache.parquet.avro.AvroReadSupport.prepareForRead(AvroReadSupport.java:142)
           at org.apache.parquet.hadoop.InternalParquetRecordReader.initialize(InternalParquetRecordReader.java: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.<init>(BaseCommand.java:344)
           at org.apache.parquet.cli.BaseCommand$1.iterator(BaseCommand.java:342)
           at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:73)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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<T> 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<T>(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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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<Object> 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. :)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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<T> 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<T>(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. 
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] theosib-amazon commented on pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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<T> 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<T>(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<T> 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<T>(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? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] theosib-amazon commented on pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] theosib-amazon commented on pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] theosib-amazon commented on pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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<T> 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<T>(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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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<Object> 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. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] theosib-amazon commented on pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
theosib-amazon commented on PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#issuecomment-1137627700

   Yes, there are some major problems with the conversions between schemas that we should turn our attention to.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] islamismailov commented on pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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 <is...@gmail.com>
       Date:   Wed May 25 19:03:33 2022 +0000
       
           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
       --- a/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java
       +++ b/parquet-avro/src/main/java/org/apache/parquet/avro/AvroSchemaConverter.java
       @@ -190,7 +190,7 @@ public class AvroSchemaConverter {
            } else if (type.equals(Schema.Type.ARRAY)) {
              if (writeOldListStructure) {
                return ConversionPatterns.listType(repetition, fieldName,
       -            convertField("array", schema.getElementType(), REPEATED, schemaPath));
       +            convertField("list", schema.getElementType(), REPEATED, schemaPath));
              } else {
                return ConversionPatterns.listOfElements(repetition, fieldName,
                    convertField(AvroWriteSupport.LIST_ELEMENT_NAME, schema.getElementType(), schemaPath));
       diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/ConversionPatterns.java b/parquet-column/src/main/java/org/apache/parquet/schema/ConversionPatterns.java
       index 8ae66f00..db1b546d 100644
       --- a/parquet-column/src/main/java/org/apache/parquet/schema/ConversionPatterns.java
       +++ b/parquet-column/src/main/java/org/apache/parquet/schema/ConversionPatterns.java
       @@ -84,8 +84,7 @@ public abstract class ConversionPatterns {
                      LogicalTypeAnnotation.mapType(),
                      new GroupType(
                              Repetition.REPEATED,
       -                      mapAlias,
       -                      LogicalTypeAnnotation.MapKeyValueTypeAnnotation.getInstance(),
       +		      "map",
                              keyType,
                              valueType)
              );
       


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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<T> 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<T>(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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] theosib-amazon commented on pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] shangxinli commented on pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
shangxinli commented on PR #957:
URL: https://github.com/apache/parquet-mr/pull/957#issuecomment-1123948507

   Can you add tests? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] islamismailov commented on pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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);`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #957: PARQUET-2069: Allow list and array record types to be compatible.

Posted by GitBox <gi...@apache.org>.
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<T> 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<T>(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<T> 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<T>(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<T> 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<T>(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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org