cdmikechen commented on code in PR #3391: URL: https://github.com/apache/hudi/pull/3391#discussion_r895724346
########## hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/avro/HoodieAvroParquetReader.java: ########## @@ -0,0 +1,104 @@ +/* + * 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.hudi.hadoop.avro; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hudi.hadoop.HoodieColumnProjectionUtils; +import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils; +import org.apache.parquet.avro.AvroReadSupport; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetInputSplit; +import org.apache.parquet.hadoop.ParquetRecordReader; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.parquet.hadoop.ParquetInputFormat.getFilter; + +public class HoodieAvroParquetReader extends RecordReader<Void, ArrayWritable> { + + private final ParquetRecordReader<GenericData.Record> parquetRecordReader; + + public HoodieAvroParquetReader(InputSplit inputSplit, Configuration conf) throws IOException { + AvroReadSupport avroReadSupport = new AvroReadSupport<>(); + // if exists read columns, we need to filter columns. + List<String> readColNames = Arrays.asList(HoodieColumnProjectionUtils.getReadColumnNames(conf)); + if (!readColNames.isEmpty()) { + // get base schema + ParquetMetadata fileFooter = Review Comment: @codope Hi~. Your consideration is consistent with my original idea. This condition only occurs when there are query columns. Based on the `parquet-avro` methods, if there is a read schema, we need to manually specify the schema of the relevant fields to correctly output data. Otherwise, schema reading will wrong, I found this problem when I checked the test exception of Ci, and then I did this. https://github.com/apache/parquet-mr/blob/a89df8f9932b6ef6633d06069e50c9b7970bebd1/parquet-avro/src/main/java/org/apache/parquet/avro/AvroReadSupport.java#L86-L139 ```java @Override public ReadContext init(Configuration configuration, Map<String, String> keyValueMetaData, MessageType fileSchema) { MessageType projection = fileSchema; Map<String, String> metadata = new LinkedHashMap<String, String>(); String requestedProjectionString = configuration.get(AVRO_REQUESTED_PROJECTION); if (requestedProjectionString != null) { Schema avroRequestedProjection = new Schema.Parser().parse(requestedProjectionString); projection = new AvroSchemaConverter(configuration).convert(avroRequestedProjection); } String avroReadSchema = configuration.get(AVRO_READ_SCHEMA); if (avroReadSchema != null) { metadata.put(AVRO_READ_SCHEMA_METADATA_KEY, avroReadSchema); } if (configuration.getBoolean(AVRO_COMPATIBILITY, AVRO_DEFAULT_COMPATIBILITY)) { metadata.put(AVRO_COMPATIBILITY, "true"); } return new ReadContext(projection, metadata); } @Override public RecordMaterializer<T> prepareForRead( Configuration configuration, Map<String, String> keyValueMetaData, MessageType fileSchema, ReadContext readContext) { Map<String, String> metadata = readContext.getReadSupportMetadata(); MessageType parquetSchema = readContext.getRequestedSchema(); Schema avroSchema; if (metadata.get(AVRO_READ_SCHEMA_METADATA_KEY) != null) { // use the Avro read schema provided by the user avroSchema = new Schema.Parser().parse(metadata.get(AVRO_READ_SCHEMA_METADATA_KEY)); } else if (keyValueMetaData.get(AVRO_SCHEMA_METADATA_KEY) != null) { // use the Avro schema from the file metadata if present avroSchema = new Schema.Parser().parse(keyValueMetaData.get(AVRO_SCHEMA_METADATA_KEY)); } else if (keyValueMetaData.get(OLD_AVRO_SCHEMA_METADATA_KEY) != null) { // use the Avro schema from the file metadata if present avroSchema = new Schema.Parser().parse(keyValueMetaData.get(OLD_AVRO_SCHEMA_METADATA_KEY)); } else { // default to converting the Parquet schema into an Avro schema avroSchema = new AvroSchemaConverter(configuration).convert(parquetSchema); } GenericData model = getDataModel(configuration); String compatEnabled = metadata.get(AvroReadSupport.AVRO_COMPATIBILITY); if (compatEnabled != null && Boolean.valueOf(compatEnabled)) { return newCompatMaterializer(parquetSchema, avroSchema, model); } return new AvroRecordMaterializer<T>(parquetSchema, avroSchema, model); } ``` -- 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: commits-unsubscr...@hudi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org