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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java:
##########
@@ -67,55 +75,72 @@
 /**
  * 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.
+ * This merge handle is used for
+ * 1. 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.
+ * 2. cow write, which takes an iterator of hoodie records, and a base file, 
and merge them,
+ *    and write to a new base file.
  */
 @NotThreadSafe
 public class FileGroupReaderBasedMergeHandle<T, I, K, O> extends 
HoodieWriteMergeHandle<T, I, K, O> {
   private static final Logger LOG = 
LoggerFactory.getLogger(FileGroupReaderBasedMergeHandle.class);
-
   private final HoodieReaderContext<T> readerContext;
-  private final CompactionOperation operation;
   private final String maxInstantTime;
-  private HoodieReadStats readStats;
   private final HoodieRecord.HoodieRecordType recordType;
   private final Option<HoodieCDCLogger> cdcLogger;
+  private final Option<CompactionOperation> operation;
+  private final Option<Iterator<HoodieRecord<T>>> recordIterator;
+  private HoodieReadStats readStats;
 
+  /**
+   * For compactor.
+   */
   public FileGroupReaderBasedMergeHandle(HoodieWriteConfig config, String 
instantTime, HoodieTable<T, I, K, O> hoodieTable,
                                          CompactionOperation operation, 
TaskContextSupplier taskContextSupplier,
+                                         Option<BaseKeyGenerator> 
keyGeneratorOpt,
                                          HoodieReaderContext<T> readerContext, 
String maxInstantTime,
                                          HoodieRecord.HoodieRecordType 
enginRecordType) {
-    super(config, instantTime, operation.getPartitionPath(), 
operation.getFileId(), hoodieTable, taskContextSupplier);
+    this(config, instantTime, hoodieTable, null, operation.getPartitionPath(), 
operation.getFileId(),
+        taskContextSupplier, keyGeneratorOpt, readerContext, maxInstantTime, 
enginRecordType, Option.of(operation));
+  }
+
+  /**
+   * For generic FG reader based merge handle.
+   */
+  public FileGroupReaderBasedMergeHandle(HoodieWriteConfig config, String 
instantTime, HoodieTable<T, I, K, O> hoodieTable,
+                                         Iterator<HoodieRecord<T>> recordItr, 
String partitionPath, String fileId,

Review Comment:
   we can change this argument to `Option<Iterator<HoodieRecord<T>>>` 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java:
##########
@@ -174,15 +199,21 @@ public void doMerge() {
     TypedProperties props = TypedProperties.copy(config.getProps());
     long maxMemoryPerCompaction = 
IOUtils.getMaxMemoryPerCompaction(taskContextSupplier, config);
     props.put(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE.key(), 
String.valueOf(maxMemoryPerCompaction));
-    Stream<HoodieLogFile> logFiles = 
operation.getDeltaFileNames().stream().map(logFileName ->
+    Stream<HoodieLogFile> logFiles = 
operation.get().getDeltaFileNames().stream().map(logFileName ->

Review Comment:
   shouldn't we do `operation.map()` or atleast we need to do 
   ```
   if (!operation.isEmpty()) 
   ```



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java:
##########
@@ -387,8 +389,16 @@ public Iterator<List<WriteStatus>> handleUpdate(String 
partitionPath, String fil
   }
 
   protected HoodieMergeHandle getUpdateHandle(String partitionPath, String 
fileId, Iterator<HoodieRecord<T>> recordItr) {
-    HoodieMergeHandle mergeHandle = 
HoodieMergeHandleFactory.create(operationType, config, instantTime, table, 
recordItr, partitionPath, fileId,
-        taskContextSupplier, keyGeneratorOpt);
+    HoodieMergeHandle mergeHandle;
+    if 
(config.getMergeHandleClassName().equals(FileGroupReaderBasedMergeHandle.class.getName()))
 {
+      HoodieReaderContext<T> readerContext = 
table.getContext().<T>getReaderContextFactory(table.getMetaClient()).getContext();
+      mergeHandle = HoodieMergeHandleFactory.create(
+          operationType, config, instantTime, table, recordItr, partitionPath, 
fileId, taskContextSupplier, keyGeneratorOpt,
+          readerContext, HoodieRecord.HoodieRecordType.SPARK);

Review Comment:
   we can't blindly choose `Spark` as the record type here. 
   HoodieWriteConfig exposes `getRecordMerger()` which inturn will expose 
`getRecordType()`.
   



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java:
##########
@@ -174,15 +199,21 @@ public void doMerge() {
     TypedProperties props = TypedProperties.copy(config.getProps());
     long maxMemoryPerCompaction = 
IOUtils.getMaxMemoryPerCompaction(taskContextSupplier, config);
     props.put(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE.key(), 
String.valueOf(maxMemoryPerCompaction));
-    Stream<HoodieLogFile> logFiles = 
operation.getDeltaFileNames().stream().map(logFileName ->
+    Stream<HoodieLogFile> logFiles = 
operation.get().getDeltaFileNames().stream().map(logFileName ->
         new HoodieLogFile(new StoragePath(FSUtils.constructAbsolutePath(
-            config.getBasePath(), operation.getPartitionPath()), 
logFileName)));
+            config.getBasePath(), operation.get().getPartitionPath()), 
logFileName)));
+    Option<Iterator<T>> engineRecordIterator = recordIterator.isEmpty()
+        ? Option.empty() : Option.of(new 
MappingIterator<>(recordIterator.get(), HoodieRecord::getData));
     // Initializes file group reader
-    try (HoodieFileGroupReader<T> fileGroupReader = 
HoodieFileGroupReader.<T>newBuilder().withReaderContext(readerContext).withHoodieTableMetaClient(hoodieTable.getMetaClient())
-        
.withLatestCommitTime(maxInstantTime).withPartitionPath(partitionPath).withBaseFileOption(Option.ofNullable(baseFileToMerge)).withLogFiles(logFiles)
-        
.withDataSchema(writeSchemaWithMetaFields).withRequestedSchema(writeSchemaWithMetaFields).withInternalSchema(internalSchemaOption).withProps(props)
+    try (HoodieFileGroupReader<T> fileGroupReader = 
HoodieFileGroupReader.<T>newBuilder()

Review Comment:
   can we instantiate the builder outside. 
   and only call either of 
   ```
   if (operation.isEmpty()) {
      fileGroupReaderBuilder.setLogFiles(logFiles);
   } else {
      fileGroupReaderBuilder.withRecordIterator(engineRecordIterator); 
   }
   
   fileGroupReaderBuilder.build(); 
   ``` 
   
   
   
   ```



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java:
##########
@@ -284,9 +340,104 @@ public void onDelete(String recordKey, T previousRecord) {
 
     }
 
+    @Override
+    public String getName() {
+      return "CdcCallBack";
+    }
+
     private GenericRecord convertOutput(T record) {
       T convertedRecord = outputConverter.get().map(converter -> record == 
null ? null : converter.apply(record)).orElse(record);
       return convertedRecord == null ? null : 
readerContext.convertToAvroRecord(convertedRecord, requestedSchema.get());
     }
   }
+
+  private static class SecondaryIndexCallback<T> implements 
BaseFileUpdateCallback<T> {
+    private final String partitionPath;
+    private final Schema writeSchemaWithMetaFields;
+    private final HoodieReaderContext<T> readerContext;
+    private final Supplier<Schema> newSchemaSupplier;
+    private final WriteStatus writeStatus;
+    private final List<HoodieIndexDefinition> secondaryIndexDefns;
+    private final Option<BaseKeyGenerator> keyGeneratorOpt;
+    private final HoodieWriteConfig config;
+
+    public SecondaryIndexCallback(String partitionPath,
+                                  Schema writeSchemaWithMetaFields,
+                                  HoodieReaderContext<T> readerContext,
+                                  Supplier<Schema> newSchemaSupplier,
+                                  WriteStatus writeStatus,
+                                  List<HoodieIndexDefinition> 
secondaryIndexDefns,
+                                  Option<BaseKeyGenerator> keyGeneratorOpt,
+                                  HoodieWriteConfig config) {
+      this.partitionPath = partitionPath;
+      this.writeSchemaWithMetaFields = writeSchemaWithMetaFields;
+      this.readerContext = readerContext;
+      this.newSchemaSupplier = newSchemaSupplier;
+      this.secondaryIndexDefns = secondaryIndexDefns;
+      this.keyGeneratorOpt = keyGeneratorOpt;
+      this.writeStatus = writeStatus;
+      this.config = config;
+    }
+
+    @Override
+    public void onUpdate(String recordKey, T previousRecord, T mergedRecord) {
+      HoodieKey hoodieKey = new HoodieKey(recordKey, partitionPath);
+      BufferedRecord<T> bufferedPrevousRecord = 
BufferedRecord.forRecordWithContext(

Review Comment:
   minor typo. 
   `bufferedPreviousRecord`. 
   "i" is missing in "previous"



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java:
##########
@@ -67,55 +75,72 @@
 /**
  * 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.
+ * This merge handle is used for
+ * 1. 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.
+ * 2. cow write, which takes an iterator of hoodie records, and a base file, 
and merge them,
+ *    and write to a new base file.
  */
 @NotThreadSafe
 public class FileGroupReaderBasedMergeHandle<T, I, K, O> extends 
HoodieWriteMergeHandle<T, I, K, O> {
   private static final Logger LOG = 
LoggerFactory.getLogger(FileGroupReaderBasedMergeHandle.class);
-
   private final HoodieReaderContext<T> readerContext;
-  private final CompactionOperation operation;
   private final String maxInstantTime;
-  private HoodieReadStats readStats;
   private final HoodieRecord.HoodieRecordType recordType;
   private final Option<HoodieCDCLogger> cdcLogger;
+  private final Option<CompactionOperation> operation;
+  private final Option<Iterator<HoodieRecord<T>>> recordIterator;
+  private HoodieReadStats readStats;
 
+  /**
+   * For compactor.
+   */
   public FileGroupReaderBasedMergeHandle(HoodieWriteConfig config, String 
instantTime, HoodieTable<T, I, K, O> hoodieTable,
                                          CompactionOperation operation, 
TaskContextSupplier taskContextSupplier,
+                                         Option<BaseKeyGenerator> 
keyGeneratorOpt,
                                          HoodieReaderContext<T> readerContext, 
String maxInstantTime,
                                          HoodieRecord.HoodieRecordType 
enginRecordType) {
-    super(config, instantTime, operation.getPartitionPath(), 
operation.getFileId(), hoodieTable, taskContextSupplier);
+    this(config, instantTime, hoodieTable, null, operation.getPartitionPath(), 
operation.getFileId(),
+        taskContextSupplier, keyGeneratorOpt, readerContext, maxInstantTime, 
enginRecordType, Option.of(operation));
+  }
+
+  /**
+   * For generic FG reader based merge handle.
+   */
+  public FileGroupReaderBasedMergeHandle(HoodieWriteConfig config, String 
instantTime, HoodieTable<T, I, K, O> hoodieTable,
+                                         Iterator<HoodieRecord<T>> recordItr, 
String partitionPath, String fileId,
+                                         TaskContextSupplier 
taskContextSupplier, Option<BaseKeyGenerator> keyGeneratorOpt,
+                                         HoodieReaderContext<T> readerContext, 
String maxInstantTime,
+                                         HoodieRecord.HoodieRecordType 
enginRecordType, Option<CompactionOperation> operation) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, 
taskContextSupplier, keyGeneratorOpt);
+    this.recordIterator = Option.ofNullable(recordItr);
+    this.operation = operation;
+
+    // Common attributes.
     this.maxInstantTime = maxInstantTime;
     this.keyToNewRecords = Collections.emptyMap();
     this.readerContext = readerContext;
-    this.operation = operation;
-    // 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;
-    if (hoodieTable.getMetaClient().getTableConfig().isCDCEnabled()) {
-      this.cdcLogger = Option.of(new HoodieCDCLogger(
-          instantTime,
-          config,
-          hoodieTable.getMetaClient().getTableConfig(),
-          partitionPath,
-          storage,
-          getWriterSchema(),
-          createLogWriter(instantTime, HoodieCDCUtils.CDC_LOGFILE_SUFFIX, 
Option.empty()),
-          IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config)));
-    } else {
-      this.cdcLogger = Option.empty();
-    }
+    this.recordType = hoodieTable.isMetadataTable() || 
HFILE.getFileExtension().equals(hoodieTable.getBaseFileExtension()) 
+        ? HoodieRecord.HoodieRecordType.AVRO : enginRecordType;
+    this.cdcLogger = 
hoodieTable.getMetaClient().getTableConfig().isCDCEnabled() 
+        ? Option.of(new HoodieCDCLogger(instantTime, config, 
hoodieTable.getMetaClient().getTableConfig(),
+            partitionPath, storage, getWriterSchema(),
+            createLogWriter(instantTime, HoodieCDCUtils.CDC_LOGFILE_SUFFIX, 
Option.empty()),
+            IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, 
config)))
+        : Option.empty();
     init(operation, this.partitionPath);
   }
 
-  private void init(CompactionOperation operation, String partitionPath) {
+  private void init(Option<CompactionOperation> operation, String 
partitionPath) {
     LOG.info("partitionPath:{}, fileId to be merged:{}", partitionPath, 
fileId);
-    this.baseFileToMerge = operation.getBaseFile(config.getBasePath(), 
operation.getPartitionPath()).orElse(null);
+

Review Comment:
   looks like you removed 
   ```
   this.baseFileToMerge = operation.getBaseFile(config.getBasePath(), 
operation.getPartitionPath()).orElse(null);
   ```
   
   So, where are we instantiating the base file now? 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java:
##########
@@ -284,9 +340,104 @@ public void onDelete(String recordKey, T previousRecord) {
 
     }
 
+    @Override
+    public String getName() {
+      return "CdcCallBack";
+    }
+
     private GenericRecord convertOutput(T record) {
       T convertedRecord = outputConverter.get().map(converter -> record == 
null ? null : converter.apply(record)).orElse(record);
       return convertedRecord == null ? null : 
readerContext.convertToAvroRecord(convertedRecord, requestedSchema.get());
     }
   }
+
+  private static class SecondaryIndexCallback<T> implements 
BaseFileUpdateCallback<T> {
+    private final String partitionPath;
+    private final Schema writeSchemaWithMetaFields;
+    private final HoodieReaderContext<T> readerContext;
+    private final Supplier<Schema> newSchemaSupplier;
+    private final WriteStatus writeStatus;
+    private final List<HoodieIndexDefinition> secondaryIndexDefns;
+    private final Option<BaseKeyGenerator> keyGeneratorOpt;
+    private final HoodieWriteConfig config;
+
+    public SecondaryIndexCallback(String partitionPath,
+                                  Schema writeSchemaWithMetaFields,
+                                  HoodieReaderContext<T> readerContext,
+                                  Supplier<Schema> newSchemaSupplier,
+                                  WriteStatus writeStatus,
+                                  List<HoodieIndexDefinition> 
secondaryIndexDefns,
+                                  Option<BaseKeyGenerator> keyGeneratorOpt,
+                                  HoodieWriteConfig config) {
+      this.partitionPath = partitionPath;
+      this.writeSchemaWithMetaFields = writeSchemaWithMetaFields;
+      this.readerContext = readerContext;
+      this.newSchemaSupplier = newSchemaSupplier;
+      this.secondaryIndexDefns = secondaryIndexDefns;
+      this.keyGeneratorOpt = keyGeneratorOpt;
+      this.writeStatus = writeStatus;
+      this.config = config;
+    }
+
+    @Override
+    public void onUpdate(String recordKey, T previousRecord, T mergedRecord) {
+      HoodieKey hoodieKey = new HoodieKey(recordKey, partitionPath);
+      BufferedRecord<T> bufferedPrevousRecord = 
BufferedRecord.forRecordWithContext(
+          previousRecord, writeSchemaWithMetaFields, readerContext, 
Option.empty(), false);
+      BufferedRecord<T> bufferedMergedRecord = 
BufferedRecord.forRecordWithContext(
+          mergedRecord, writeSchemaWithMetaFields, readerContext, 
Option.empty(), false);
+      SecondaryIndexStreamingTracker.trackSecondaryIndexStats(
+          hoodieKey,
+          Option.of(readerContext.constructHoodieRecord(bufferedMergedRecord)),

Review Comment:
   So, internally (w/n FG reader), we translate HoodieRecord to engine specific 
representation and wrap it using BufferedRecord (FG reader processing). and 
here, we get notifications on engine specific record. We create BufferedRecord 
and then construct HoodieRecord from bufferedRecord. 
   
   why not directly create HoodieRecord from engine specific representation 
here.
   



##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/InputBasedFileGroupRecordBuffer.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.common.table.read;
+
+import org.apache.hudi.common.config.RecordMergeMode;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.model.DeleteRecord;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.PartialUpdateMode;
+import org.apache.hudi.common.table.log.KeySpec;
+import org.apache.hudi.common.table.log.block.HoodieDataBlock;
+import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieNotSupportedException;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Iterator;
+
+public class InputBasedFileGroupRecordBuffer<T> extends 
KeyBasedFileGroupRecordBuffer<T> {

Review Comment:
   java docs



##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/UpdateProcessor.java:
##########
@@ -99,13 +107,29 @@ public T processUpdate(String recordKey, T previousRecord, 
T currentRecord, bool
       T result = delegate.processUpdate(recordKey, previousRecord, 
currentRecord, isDelete);
 
       if (isDelete) {
-        callback.onDelete(recordKey, previousRecord);
+        invokeCallbacks(callback -> callback.onDelete(recordKey, 
previousRecord));
       } else if (previousRecord != null && previousRecord != currentRecord) {
-        callback.onUpdate(recordKey, previousRecord, currentRecord);
+        invokeCallbacks(callback -> callback.onUpdate(recordKey, 
previousRecord, currentRecord));
       } else {
-        callback.onInsert(recordKey, currentRecord);
+        invokeCallbacks(callback -> callback.onInsert(recordKey, 
currentRecord));
       }
+
       return result;
     }
+
+    private void invokeCallbacks(CallbackInvoker<T> invoker) {
+      for (BaseFileUpdateCallback<T> callback : callbacks) {
+        try {
+          invoker.invoke(callback);
+        } catch (Exception e) {
+          LOG.error(String.format("Callback %s failed: ", callback.getName()), 
e);

Review Comment:
   can we use {} 



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