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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##########
@@ -985,9 +985,10 @@ public void performTableServices(Option<String> 
inFlightInstantTimestamp) {
 
       // Do timeline validation before scheduling compaction/logcompaction 
operations.
       if (validateTimelineBeforeSchedulingCompaction(inFlightInstantTimestamp, 
latestDeltacommitTime)) {
+        LOG.info("No inflight commits present in either of the timelines. "
+            + "Proceeding to check compaction.");

Review Comment:
   Do we need a log like this, the compaction would put many loggins anyway.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##########
@@ -1021,17 +1023,46 @@ private void 
runPendingTableServicesOperations(BaseHoodieWriteClient writeClient
    * deltacommit.
    */
   protected void compactIfNecessary(BaseHoodieWriteClient writeClient, String 
latestDeltacommitTime) {
+
+    // Check if there are any pending compaction or log compaction instants in 
the timeline.
+    // If pending compact/logcompaction operations are found abort scheduling 
new compaction/logcompaction operations.
+    Option<HoodieInstant> pendingLogCompactionInstant =
+        
metadataMetaClient.getActiveTimeline().filterPendingLogCompactionTimeline().firstInstant();

Review Comment:
   Wondering how could this happen, all the compactions are inline for MDT 
right?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java:
##########
@@ -111,6 +112,10 @@ public static HoodieWriteConfig createMetadataWriteConfig(
             // deltacommits having corresponding completed commits. Therefore, 
we need to compact all fileslices of all
             // partitions together requiring UnBoundedCompactionStrategy.
             .withCompactionStrategy(new UnBoundedCompactionStrategy())
+            // Check if log compaction is enabled, this is needed for tables 
with lot of records.
+            .withLogCompactionEnabled(writeConfig.isLogCompactionEnabled())
+            // This config is only used if enableLogCompactionForMetadata is 
set.

Review Comment:
   Yeah, log compaction for MDT and DT should not share the same config option 
I think.



##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java:
##########
@@ -233,48 +233,49 @@ void testSyncMetadataTable() throws Exception {
     assertThat(completedTimeline.lastInstant().get().getTimestamp(), 
startsWith(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP));
 
     // test metadata table compaction
-    // write another 4 commits
-    for (int i = 1; i < 5; i++) {
+    // write another 9 commits to trigger compaction twice. Since default 
clean version to retain is 2.

Review Comment:
   Why we need to change the clean strategy then?



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