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


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkFileGroupReaderBasedMergeHandle.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+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.HoodieKey;
+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.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import 
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieReadStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+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.storage.hadoop.HadoopStorageConfiguration;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+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.Properties;
+import java.util.stream.Collectors;
+
+import static 
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+/**
+ * A merge handle implementation based on the {@link HoodieFileGroupReader}.
+ * <p>
+ * This merge handle is used for compaction on Spark, 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.
+ */
+@NotThreadSafe
+public class HoodieSparkFileGroupReaderBasedMergeHandle<T, I, K, O> extends 
HoodieMergeHandle<T, I, K, O> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieSparkFileGroupReaderBasedMergeHandle.class);
+
+  protected HoodieReaderContext readerContext;
+  protected FileSlice fileSlice;
+  protected Configuration conf;
+
+  public HoodieSparkFileGroupReaderBasedMergeHandle(HoodieWriteConfig config, 
String instantTime, HoodieTable<T, I, K, O> hoodieTable,
+                                                    CompactionOperation 
operation, TaskContextSupplier taskContextSupplier,
+                                                    Option<BaseKeyGenerator> 
keyGeneratorOpt,
+                                                    HoodieReaderContext 
readerContext, Configuration conf) {
+    super(config, instantTime, operation.getPartitionPath(), 
operation.getFileId(), hoodieTable, taskContextSupplier);
+    this.keyToNewRecords = Collections.emptyMap();
+    this.readerContext = readerContext;
+    this.conf = conf;
+    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.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;
+  }
+
+  private void init(CompactionOperation operation, String partitionPath, 
Option<HoodieBaseFile> baseFileToMerge) {

Review Comment:
   lets add some java docs as to why this differs from the base 
HoodieMergeHandle



##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java:
##########
@@ -237,7 +241,8 @@ protected Option<Pair<T, Map<String, Object>>> 
doProcessNextDataRecord(T record,
             Comparable incomingOrderingValue = readerContext.getOrderingValue(
                 Option.of(record), metadata, readerSchema, orderingFieldName, 
orderingFieldTypeOpt, orderingFieldDefault);
             if (incomingOrderingValue.compareTo(existingOrderingValue) > 0) {
-              return Option.of(Pair.of(record, metadata));
+              return Option.of(Pair.of(isDeleteRecord(Option.of(record), 
(Schema) metadata.get(INTERNAL_META_SCHEMA))

Review Comment:
   I feel we should not have 2 ways of handling deletes within 
doProcessNextDataRecord. may be, we should process records from data block, and 
call into processNextDeletedRecord() if we detect a deleted record. 
   
   that way, we have only one way of handling a deleted record. 
   
   and processNextDataRecord will only take in valid records. we can avoid 
calling isDeleteRecord(Option.of(record)) within processNextDataRecord. 
   



##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java:
##########
@@ -408,22 +413,20 @@ protected Option<T> merge(Option<T> older, Map<String, 
Object> olderInfoMap,
     } else {
       switch (recordMergeMode) {
         case COMMIT_TIME_ORDERING:
-          return newer;
+          return isDeleteRecord(newer, (Schema) 
newerInfoMap.get(INTERNAL_META_SCHEMA)) ? Option.empty() : newer;
         case EVENT_TIME_ORDERING:
-          Comparable oldOrderingValue = readerContext.getOrderingValue(
-              older, olderInfoMap, readerSchema, orderingFieldName, 
orderingFieldTypeOpt, orderingFieldDefault);
-          if (isDeleteRecordWithNaturalOrder(older, oldOrderingValue)) {
-            return newer;
-          }
           Comparable newOrderingValue = readerContext.getOrderingValue(
               newer, newerInfoMap, readerSchema, orderingFieldName, 
orderingFieldTypeOpt, orderingFieldDefault);
           if (isDeleteRecordWithNaturalOrder(newer, newOrderingValue)) {
             return Option.empty();
           }
-          if (oldOrderingValue.compareTo(newOrderingValue) > 0) {
-            return older;
+          Comparable oldOrderingValue = readerContext.getOrderingValue(

Review Comment:
   Q: even applicable before this patch. 
   if old record has a valid ordering value. and if new one has natural 
ordering value and is deleted, whats the expectation, when we merge both these?
   



##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java:
##########
@@ -237,7 +241,8 @@ protected Option<Pair<T, Map<String, Object>>> 
doProcessNextDataRecord(T record,
             Comparable incomingOrderingValue = readerContext.getOrderingValue(
                 Option.of(record), metadata, readerSchema, orderingFieldName, 
orderingFieldTypeOpt, orderingFieldDefault);
             if (incomingOrderingValue.compareTo(existingOrderingValue) > 0) {
-              return Option.of(Pair.of(record, metadata));
+              return Option.of(Pair.of(isDeleteRecord(Option.of(record), 
(Schema) metadata.get(INTERNAL_META_SCHEMA))

Review Comment:
   but lets try to land this patch. and do an immediate follow up.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java:
##########
@@ -408,22 +413,20 @@ protected Option<T> merge(Option<T> older, Map<String, 
Object> olderInfoMap,
     } else {
       switch (recordMergeMode) {
         case COMMIT_TIME_ORDERING:
-          return newer;
+          return isDeleteRecord(newer, (Schema) 
newerInfoMap.get(INTERNAL_META_SCHEMA)) ? Option.empty() : newer;
         case EVENT_TIME_ORDERING:
-          Comparable oldOrderingValue = readerContext.getOrderingValue(
-              older, olderInfoMap, readerSchema, orderingFieldName, 
orderingFieldTypeOpt, orderingFieldDefault);
-          if (isDeleteRecordWithNaturalOrder(older, oldOrderingValue)) {
-            return newer;
-          }
           Comparable newOrderingValue = readerContext.getOrderingValue(
               newer, newerInfoMap, readerSchema, orderingFieldName, 
orderingFieldTypeOpt, orderingFieldDefault);
           if (isDeleteRecordWithNaturalOrder(newer, newOrderingValue)) {
             return Option.empty();
           }
-          if (oldOrderingValue.compareTo(newOrderingValue) > 0) {
-            return older;
+          Comparable oldOrderingValue = readerContext.getOrderingValue(

Review Comment:
   I get it. we will treat it as deleted. 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkFileGroupReaderBasedMergeHandle.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+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.HoodieKey;
+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.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import 
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieReadStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+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.storage.hadoop.HadoopStorageConfiguration;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+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.Properties;
+import java.util.stream.Collectors;
+
+import static 
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+/**
+ * A merge handle implementation based on the {@link HoodieFileGroupReader}.
+ * <p>
+ * This merge handle is used for compaction on Spark, 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.
+ */
+@NotThreadSafe
+public class HoodieSparkFileGroupReaderBasedMergeHandle<T, I, K, O> extends 
HoodieMergeHandle<T, I, K, O> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieSparkFileGroupReaderBasedMergeHandle.class);
+
+  protected HoodieReaderContext readerContext;
+  protected FileSlice fileSlice;
+  protected Configuration conf;
+
+  public HoodieSparkFileGroupReaderBasedMergeHandle(HoodieWriteConfig config, 
String instantTime, HoodieTable<T, I, K, O> hoodieTable,
+                                                    CompactionOperation 
operation, TaskContextSupplier taskContextSupplier,
+                                                    Option<BaseKeyGenerator> 
keyGeneratorOpt,
+                                                    HoodieReaderContext 
readerContext, Configuration conf) {
+    super(config, instantTime, operation.getPartitionPath(), 
operation.getFileId(), hoodieTable, taskContextSupplier);
+    this.keyToNewRecords = Collections.emptyMap();
+    this.readerContext = readerContext;
+    this.conf = conf;
+    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.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;
+  }
+
+  private void init(CompactionOperation operation, String partitionPath, 
Option<HoodieBaseFile> baseFileToMerge) {
+    LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" + 
fileId);
+    this.baseFileToMerge = baseFileToMerge.orElse(null);
+    this.writtenRecordKeys = new HashSet<>();
+    writeStatus.setStat(new HoodieWriteStat());
+    writeStatus.getStat().setTotalLogSizeCompacted(
+        
operation.getMetrics().get(CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue());
+    try {
+      Option<String> latestValidFilePath = Option.empty();
+      if (baseFileToMerge.isPresent()) {
+        latestValidFilePath = Option.of(baseFileToMerge.get().getFileName());
+        
writeStatus.getStat().setPrevCommit(baseFileToMerge.get().getCommitTime());
+        // At the moment, we only support SI for overwrite with latest 
payload. So, we don't need to embed entire file slice here.
+        // HUDI-8518 will be taken up to fix it for any payload during which 
we might require entire file slice to be set here.
+        // Already AppendHandle adds all logs file from current file slice to 
HoodieDeltaWriteStat.
+        writeStatus.getStat().setPrevBaseFile(latestValidFilePath.get());
+      } else {
+        writeStatus.getStat().setPrevCommit(HoodieWriteStat.NULL_COMMIT);
+      }
+
+      HoodiePartitionMetadata partitionMetadata = new 
HoodiePartitionMetadata(storage, instantTime,
+          new StoragePath(config.getBasePath()),
+          FSUtils.constructAbsolutePath(config.getBasePath(), partitionPath),
+          hoodieTable.getPartitionMetafileFormat());
+      partitionMetadata.trySave();
+
+      String newFileName = FSUtils.makeBaseFileName(instantTime, writeToken, 
fileId, hoodieTable.getBaseFileExtension());

Review Comment:
   as a follow up, can we try to reduce code duplication 



##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieBaseFileGroupRecordBuffer.java:
##########
@@ -559,4 +562,18 @@ private boolean isDeleteRecordWithNaturalOrder(Option<T> 
rowOption,
                                                  Comparable orderingValue) {
     return rowOption.isEmpty() && orderingValue.equals(orderingFieldDefault);
   }
+
+  private boolean isDeleteRecord(Option<T> record, Schema schema) {
+    if (record.isEmpty()) {
+      return true;
+    }
+
+    Object operation = readerContext.getValue(record.get(), schema, 
OPERATION_METADATA_FIELD);
+    if (operation != null && 
HoodieOperation.isDeleteRecord(operation.toString())) {
+      return true;
+    }
+
+    Object deleteMarker = readerContext.getValue(record.get(), schema, 
HOODIE_IS_DELETED_FIELD);
+    return deleteMarker instanceof Boolean && (boolean) deleteMarker;

Review Comment:
   do we have a follow up ticket to support custom delete marker? 
   do we know if the features works in the first place in officially release 
hudi versions ?



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