-
Notifications
You must be signed in to change notification settings - Fork 1.5k
PARQUET-2069: Allow list and array record types to be compatible. #957
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
Open
theosib-amazon
wants to merge
5
commits into
apache:master
Choose a base branch
from
theosib-amazon:PARQUET-2069
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
b8c7c15
Fix PARQUET-2069: Allow list and array record types to be compatible.
theosib-amazon 6c9f2d4
Update AvroRecordConverter.java
theosib-amazon 325fa1c
Fix PARQUET-2069 and related bugs
theosib-amazon a067dd6
Create TestArrayListCompatibility.java
theosib-amazon 552f9ef
Update AvroReadSupport.java
theosib-amazon File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,8 +25,10 @@ | |
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.util.ReflectionUtils; | ||
| import org.apache.parquet.hadoop.api.ReadSupport; | ||
| import org.apache.parquet.io.InvalidRecordException; | ||
| import org.apache.parquet.io.api.RecordMaterializer; | ||
| import org.apache.parquet.schema.MessageType; | ||
| import org.apache.log4j.Logger; | ||
|
|
||
| /** | ||
| * Avro implementation of {@link ReadSupport} for avro generic, specific, and | ||
|
|
@@ -36,6 +38,7 @@ | |
| * @param <T> the Java type of records created by this ReadSupport | ||
| */ | ||
| public class AvroReadSupport<T> extends ReadSupport<T> { | ||
| static Logger log = Logger.getLogger(AvroReadSupport.class.getName()); | ||
|
|
||
| public static String AVRO_REQUESTED_PROJECTION = "parquet.avro.projection"; | ||
| private static final String AVRO_READ_SCHEMA = "parquet.avro.read.schema"; | ||
|
|
@@ -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); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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)? |
||
|
|
||
| if (compatEnabled != null && Boolean.valueOf(compatEnabled)) { | ||
| return newCompatMaterializer(parquetSchema, avroSchema, model); | ||
| } | ||
| return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model); | ||
| } | ||
| return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model); | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
|
|
||
46 changes: 46 additions & 0 deletions
46
parquet-avro/src/test/java/org/apache/parquet/avro/TestArrayListCompatibility.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,46 @@ | ||
| /** | ||
| * 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; | ||
| firstRecord = (GenericData.Record) parquetReader.read(); | ||
| if (firstRecord == null) { | ||
| throw new IOException("Can't process empty Parquet file"); | ||
| } | ||
| firstRecord.getSchema().toString(true); | ||
| } | ||
|
|
||
| } |
Binary file not shown.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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)?
There was a problem hiding this comment.
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.