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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java:
##########
@@ -317,33 +318,40 @@ protected HoodieWriteMetadata<O> compact(HoodieTable<?, 
I, ?, T> table, String c
     }
     compactionTimer = metrics.getCompactionCtx();
     HoodieWriteMetadata<T> writeMetadata = table.compact(context, 
compactionInstantTime);
-    HoodieWriteMetadata<O> compactionWriteMetadata = 
convertToOutputMetadata(writeMetadata);
+    HoodieWriteMetadata<T> processedWriteMetadata = 
processWriteMetadata(table, writeMetadata, compactionInstantTime);
+    HoodieWriteMetadata<O> compactionWriteMetadata = 
convertToOutputMetadata(processedWriteMetadata);
     if (shouldComplete) {
       commitCompaction(compactionInstantTime, compactionWriteMetadata, 
Option.of(table));
     }
     return compactionWriteMetadata;
   }
 
+  protected HoodieWriteMetadata<T> processWriteMetadata(HoodieTable table, 
HoodieWriteMetadata<T> writeMetadata, String instantTime) {
+    return writeMetadata;
+  }
+
   public void commitCompaction(String compactionInstantTime, 
HoodieWriteMetadata<O> compactionWriteMetadata, Option<HoodieTable> tableOpt) {
     // dereferencing the write dag for compaction for the first time.
-    List<HoodieWriteStat> writeStats = 
triggerWritesAndFetchWriteStats(compactionWriteMetadata);
+    Pair<List<HoodieWriteStat>, List<HoodieWriteStat>> 
dataTableAndMetadataTableHoodieWriteStats = 
triggerWritesAndFetchWriteStats(compactionWriteMetadata);
     // Fetch commit metadata from HoodieWriteMetadata and update 
HoodieWriteStat
-    
CommonClientUtils.stitchCompactionHoodieWriteStats(compactionWriteMetadata, 
writeStats);
+    
CommonClientUtils.stitchCompactionHoodieWriteStats(compactionWriteMetadata, 
dataTableAndMetadataTableHoodieWriteStats.getKey());
     metrics.emitCompactionCompleted();
 
     HoodieTable table = tableOpt.orElseGet(() -> createTable(config, 
context.getStorageConf()));
-    completeCompaction(compactionWriteMetadata.getCommitMetadata().get(), 
table, compactionInstantTime);
+    completeCompaction(compactionWriteMetadata.getCommitMetadata().get(), 
table, compactionInstantTime, 
dataTableAndMetadataTableHoodieWriteStats.getValue());
   }
 
   /**
    * The API triggers the data write and fetches the corresponding write stats 
using the write metadata.
+   * When streaming writes to metadata table is enabled, writes to metadata 
table is expected to be triggered here and the List of {@link HoodieWriteStat} 
to be returned
+   * as part of this call.
    */
-  protected abstract List<HoodieWriteStat> 
triggerWritesAndFetchWriteStats(HoodieWriteMetadata<O> writeMetadata);
+  protected abstract Pair<List<HoodieWriteStat>, List<HoodieWriteStat>> 
triggerWritesAndFetchWriteStats(HoodieWriteMetadata<O> writeMetadata);
 
   /**
    * Commit Compaction and track metrics.
    */
-  protected void completeCompaction(HoodieCommitMetadata metadata, HoodieTable 
table, String compactionCommitTime) {
+  protected void completeCompaction(HoodieCommitMetadata metadata, HoodieTable 
table, String compactionCommitTime, List<HoodieWriteStat> 
metadataWriteStatsSoFar) {

Review Comment:
   hmmm. not sure if current suits here well. 
   `metadataWriteStatsSoFar` signifies that this represents write stats to 
metadata written so far. 
   or we can name `partialMetadataWriteStats` 



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