Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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";
Expand Down Expand Up @@ -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) {
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.

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


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")
Expand Down
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.