nsivabalan commented on code in PR #13411:
URL: https://github.com/apache/hudi/pull/13411#discussion_r2153249062


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java:
##########
@@ -20,81 +20,80 @@
 package org.apache.hudi.io;
 
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.HoodieMemoryConfig;
+import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.engine.HoodieReaderContext;
 import org.apache.hudi.common.engine.TaskContextSupplier;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.CompactionOperation;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieBaseFile;
-import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodiePartitionMetadata;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.table.read.HoodieFileGroupReader;
 import org.apache.hudi.common.table.read.HoodieReadStats;
 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.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
 import org.apache.hudi.io.storage.HoodieFileWriterFactory;
-import org.apache.hudi.keygen.BaseKeyGenerator;
 import org.apache.hudi.storage.StoragePath;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.concurrent.NotThreadSafe;
+
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
-import java.util.stream.Collectors;
+import java.util.Map;
+
+import static 
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
 
 /**
- * A base merge handle implementation based on the {@link 
HoodieFileGroupReader}.
+ * A merge handle implementation based on the {@link HoodieFileGroupReader}.
  * <p>
  * This merge handle is used for compaction, which passes a file slice from the
  * compaction operation of a single file group to a file group reader, get an 
iterator of
  * the records, and writes the records to a new base file.
  */
-public abstract class BaseFileGroupReaderBasedMergeHandle<T, I, K, O> extends 
HoodieMergeHandle<T, I, K, O> {
-  private static final Logger LOG = 
LoggerFactory.getLogger(BaseFileGroupReaderBasedMergeHandle.class);
-
-  protected HoodieReaderContext<T> readerContext;
-  protected FileSlice fileSlice;
-  protected HoodieReadStats readStats;
-
-  public BaseFileGroupReaderBasedMergeHandle(HoodieWriteConfig config, String 
instantTime, HoodieTable<T, I, K, O> hoodieTable,
-                                             CompactionOperation operation, 
TaskContextSupplier taskContextSupplier,
-                                             Option<BaseKeyGenerator> 
keyGeneratorOpt,
-                                             HoodieReaderContext<T> 
readerContext) {
+@NotThreadSafe
+public class FileGroupReaderBasedMergeHandle<T, I, K, O> extends 
HoodieMergeHandle<T, I, K, O> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(FileGroupReaderBasedMergeHandle.class);
+
+  private final HoodieReaderContext<T> readerContext;
+  private final FileSlice fileSlice;
+  private final CompactionOperation operation;
+  private final String maxInstantTime;
+  private HoodieReadStats readStats;
+  private final HoodieRecord.HoodieRecordType recordType;
+
+  public FileGroupReaderBasedMergeHandle(HoodieWriteConfig config, String 
instantTime, HoodieTable<T, I, K, O> hoodieTable,
+                                         FileSlice fileSlice, 
CompactionOperation operation, TaskContextSupplier taskContextSupplier,
+                                         HoodieReaderContext<T> readerContext, 
String maxInstantTime,
+                                         HoodieRecord.HoodieRecordType 
enginRecordType) {
     super(config, instantTime, operation.getPartitionPath(), 
operation.getFileId(), hoodieTable, taskContextSupplier);
+    this.maxInstantTime = maxInstantTime;
     this.keyToNewRecords = Collections.emptyMap();
     this.readerContext = readerContext;
-    Option<HoodieBaseFile> baseFileOpt =
-        operation.getBaseFile(config.getBasePath(), 
operation.getPartitionPath());
-    List<HoodieLogFile> logFiles = 
operation.getDeltaFileNames().stream().map(p ->
-            new HoodieLogFile(new StoragePath(FSUtils.constructAbsolutePath(
-                config.getBasePath(), operation.getPartitionPath()), p)))
-        .collect(Collectors.toList());
-    this.fileSlice = new FileSlice(
-        operation.getFileGroupId(),
-        operation.getBaseInstantTime(),
-        baseFileOpt.isPresent() ? baseFileOpt.get() : null,
-        logFiles);
+    this.fileSlice = fileSlice;
+    this.operation = operation;
     this.preserveMetadata = true;
-    init(operation, this.partitionPath, baseFileOpt);
-    validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields());
-  }
-
-  private void validateAndSetAndKeyGenProps(Option<BaseKeyGenerator> 
keyGeneratorOpt, boolean populateMetaFields) {
-    ValidationUtils.checkArgument(populateMetaFields == 
!keyGeneratorOpt.isPresent());
-    this.keyGeneratorOpt = keyGeneratorOpt;
+    // If the table is a metadata table or the base file is an HFile, we use 
AVRO record type, otherwise we use the engine record type.
+    this.recordType = hoodieTable.isMetadataTable() || 
HFILE.getFileExtension().equals(hoodieTable.getBaseFileExtension()) ? 
HoodieRecord.HoodieRecordType.AVRO : enginRecordType;

Review Comment:
   sg. if this is the only use of deducing record type, its fine. 
   



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