Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

PARQUET-2069: Allow list and array record types to be compatible. #957

Open
wants to merge 5 commits into
base: master
Choose a base branch
from

Conversation

theosib-amazon
Copy link
Contributor

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.

@shangxinli
Copy link
Contributor

Can you add tests?

Changed formatting of IF statements.
@theosib-amazon
Copy link
Contributor Author

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.

@theosib-amazon
Copy link
Contributor Author

theosib-amazon commented May 13, 2022

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, imitating the way I did it in my own reader. 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(s) 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.

Sometimes the avro schema is bad. This fix will attempt to use
the avro schema, but if that fails, it'll fall back to converting
the parquet schema.
@theosib-amazon
Copy link
Contributor Author

theosib-amazon commented May 16, 2022

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

@islamismailov
Copy link

islamismailov commented May 24, 2022

@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>>?

Warning, Avro schema doesn't match Parquet schema, falling back to conversion: java.lang.ClassCastException: required binary element (STRING) is not a group
Unknown error
java.lang.RuntimeException: Failed on record 0
at org.apache.parquet.cli.commands.CatCommand.run(CatCommand.java:86)
at org.apache.parquet.cli.Main.run(Main.java:157)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java: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: org.apache.parquet.io.ParquetDecodingException: Can not read value at 0 in block -1 in file dbg.parquet
at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:264)
at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:132)
at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:136)
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)
... 9 more
Caused by: java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
at java.util.ArrayList.rangeCheck(ArrayList.java:659)
at java.util.ArrayList.get(ArrayList.java:435)
at org.apache.parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:101)
at org.apache.parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:101)
at org.apache.parquet.io.PrimitiveColumnIO.getFirst(PrimitiveColumnIO.java:97)
at org.apache.parquet.io.PrimitiveColumnIO.isFirst(PrimitiveColumnIO.java:92)
at org.apache.parquet.io.RecordReaderImplementation.(RecordReaderImplementation.java:278)
at org.apache.parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:147)
at org.apache.parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:109)
at org.apache.parquet.filter2.compat.FilterCompat$NoOpFilter.accept(FilterCompat.java:177)
at org.apache.parquet.io.MessageColumnIO.getRecordReader(MessageColumnIO.java:109)
at org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:146)
at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:235)
... 15 more

@theosib-amazon
Copy link
Contributor Author

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

@islamismailov
Copy link

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

@theosib-amazon
Copy link
Contributor Author

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

@islamismailov
Copy link

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 apache/iceberg#3309

Link to the original issue: apache/iceberg#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 <islamismailov@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)
       );

@theosib-amazon
Copy link
Contributor Author

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?

try {
firstRecord = (GenericData.Record) parquetReader.read();
} catch (Exception x) {
x.printStackTrace();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

}
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());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Any reason we don't use Log4j?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oversight on my part.

return newCompatMaterializer(parquetSchema, avroSchema, model);
}
return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model);
} catch (InvalidRecordException | ClassCastException e) {
Copy link
Contributor

@shangxinli shangxinli Jun 18, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Allow any exception from parquetReader.read() propagate up without intermediate catch.
return newCompatMaterializer(parquetSchema, avroSchema, model);
}
return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model);
} catch (InvalidRecordException | ClassCastException e) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

return newCompatMaterializer(parquetSchema, avroSchema, model);
}
return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model);
} catch (InvalidRecordException | ClassCastException e) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

} 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);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants