danny0405 commented on code in PR #13078:
URL: https://github.com/apache/hudi/pull/13078#discussion_r2049891008


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FlinkRowDataReaderContext.java:
##########
@@ -0,0 +1,257 @@
+/*
+ * 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.table.format;
+
+import org.apache.hudi.client.model.CommitTimeFlinkRecordMerger;
+import org.apache.hudi.client.model.EventTimeFlinkRecordMerger;
+import org.apache.hudi.client.model.HoodieFlinkRecord;
+import org.apache.hudi.common.config.RecordMergeMode;
+import org.apache.hudi.common.engine.EngineType;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieEmptyRecord;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordMerger;
+import org.apache.hudi.common.util.HoodieRecordUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.ClosableIterator;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieValidationException;
+import org.apache.hudi.source.ExpressionPredicates.Predicate;
+import org.apache.hudi.storage.HoodieStorage;
+import org.apache.hudi.storage.StorageConfiguration;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.util.AvroSchemaConverter;
+import org.apache.hudi.util.AvroToRowDataConverters;
+import org.apache.hudi.util.FlinkRowProjection;
+import org.apache.hudi.util.RowDataAvroQueryContexts;
+import org.apache.hudi.util.RowDataUtils;
+import org.apache.hudi.util.AvroConverterUtils;
+import org.apache.hudi.util.SchemaEvolvingRowDataProjection;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.JoinedRowData;
+import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.UnaryOperator;
+import java.util.stream.Collectors;
+
+import static 
org.apache.hudi.common.config.HoodieReaderConfig.RECORD_MERGE_IMPL_CLASSES_WRITE_CONFIG_KEY;
+import static org.apache.hudi.common.model.HoodieRecord.DEFAULT_ORDERING_VALUE;
+import static 
org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD;
+
+/**
+ * Implementation of {@link HoodieReaderContext} to read {@link RowData}s from 
base files or
+ * log files with Flink parquet reader.
+ */
+public class FlinkRowDataReaderContext extends HoodieReaderContext<RowData> {
+  private final List<Predicate> predicates;
+  private final InternalSchemaManager internalSchemaManager;
+  private RowDataSerializer rowDataSerializer;
+
+  public FlinkRowDataReaderContext(
+      StorageConfiguration<?> storageConfiguration,
+      InternalSchemaManager internalSchemaManager,
+      List<Predicate> predicates) {
+    super(storageConfiguration);
+    this.internalSchemaManager = internalSchemaManager;
+    this.predicates = predicates;
+  }
+
+  @Override
+  public ClosableIterator<RowData> getFileRecordIterator(
+      StoragePath filePath,
+      long start,
+      long length,
+      Schema dataSchema,
+      Schema requiredSchema,
+      HoodieStorage storage) throws IOException {
+    boolean isLogFile = FSUtils.isLogFile(filePath);
+    // disable schema evolution in fileReader if it's log file, since schema 
evolution for log file is handled in `FileGroupRecordBuffer`
+    InternalSchemaManager schemaManager = isLogFile ? 
InternalSchemaManager.DISABLED : internalSchemaManager;
+    RowDataFileReaderFactories.Factory readerFactory = 
RowDataFileReaderFactories.getFactory(HoodieFileFormat.PARQUET);
+    RowDataFileReader fileReader = 
readerFactory.createFileReader(schemaManager, getStorageConfiguration());
+
+    List<String> fieldNames = 
dataSchema.getFields().stream().map(Schema.Field::name).collect(Collectors.toList());
+    List<DataType> fieldTypes = dataSchema.getFields().stream().map(
+        f -> 
AvroSchemaConverter.convertToDataType(f.schema())).collect(Collectors.toList());

Review Comment:
   maybe convert to data type in one shot and `getChildren`



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to