cdmikechen commented on code in PR #3391:
URL: https://github.com/apache/hudi/pull/3391#discussion_r895728574


##########
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 =
+              ParquetFileReader.readFooter(conf, ((ParquetInputSplit) 
inputSplit).getPath(), ParquetMetadataConverter.NO_FILTER);
+      MessageType messageType = fileFooter.getFileMetaData().getSchema();
+      Schema baseSchema = new AvroSchemaConverter(conf).convert(messageType);
+      // filter schema for reading
+      final Schema filterSchema = Schema.createRecord(baseSchema.getName(),
+              baseSchema.getDoc(), baseSchema.getNamespace(), 
baseSchema.isError(),
+              baseSchema.getFields().stream()
+                      .filter(f -> readColNames.contains(f.name()))
+                      .map(f -> new Schema.Field(f.name(), f.schema(), 
f.doc(), f.defaultVal()))
+              .collect(Collectors.toList()));
+      avroReadSupport.setAvroReadSchema(conf, filterSchema);
+      avroReadSupport.setRequestedProjection(conf, filterSchema);
+    }
+    parquetRecordReader = new ParquetRecordReader<>(avroReadSupport, 
getFilter(conf));
+  }
+
+  @Override
+  public void initialize(InputSplit split, TaskAttemptContext context) throws 
IOException, InterruptedException {
+    parquetRecordReader.initialize(split, context);
+  }
+
+  @Override
+  public boolean nextKeyValue() throws IOException, InterruptedException {
+    return parquetRecordReader.nextKeyValue();
+  }
+
+  @Override
+  public Void getCurrentKey() throws IOException, InterruptedException {
+    return parquetRecordReader.getCurrentKey();
+  }
+
+  @Override
+  public ArrayWritable getCurrentValue() throws IOException, 
InterruptedException {
+    GenericRecord record = parquetRecordReader.getCurrentValue();
+    return (ArrayWritable) 
HoodieRealtimeRecordReaderUtils.avroToArrayWritable(record, record.getSchema(), 
true);

Review Comment:
   @codope 
   If we use this class, it means that we need to use the timestamp type, so I 
directly set it to true here.



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

Reply via email to