You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/01/07 06:28:28 UTC

[GitHub] [hudi] Karl-WangSK opened a new pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Karl-WangSK opened a new pull request #2260:
URL: https://github.com/apache/hudi/pull/2260


   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contributing.html before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   Schedule compaction based on time elapsed
   
   ## Brief change log
   
   GH : https://github.com/apache/hudi/issues/2229
   It would be helpful to introduce configuration to schedule compaction based on time elapsed since last scheduled compaction.
   
   ## Verify this pull request
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   ## Committer checklist
   
    - [x] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] wangxianghu commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
wangxianghu commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r548791159



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -129,6 +130,7 @@ public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig
     this.metrics = new HoodieMetrics(config, config.getTableName());
     this.rollbackPending = rollbackPending;
     this.index = createIndex(writeConfig);
+    this.initialTime = HoodieActiveTimeline.createNewInstantTime();

Review comment:
       How about mark this `initialTime` as lastCompaction time, then we can gei it from timeline. 
   then, when we get a new commit, we can check the interval between these two timestamps to decide whether execute compact or not.
   in this way:
   1. there is no need to update it additionally
   2. It would be more accurate.(Time elapsed since the last compact)




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (8d37749) into [master](https://codecov.io/gh/apache/hudi/commit/4d05680038752077ceaebef261b66a5afc761e10?el=desc) (4d05680) will **decrease** coverage by `43.12%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     53.53%   10.41%   -43.13%     
   + Complexity     2770       48     -2722     
   =============================================
     Files           348       50      -298     
     Lines         16109     1777    -14332     
     Branches       1643      211     -1432     
   =============================================
   - Hits           8624      185     -8439     
   + Misses         6786     1579     -5207     
   + Partials        699       13      -686     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudispark | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.41% <ø> (-59.69%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | ... and [324 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (3639b90) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **decrease** coverage by `42.16%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     52.20%   10.04%   -42.17%     
   + Complexity     2659       48     -2611     
   =============================================
     Files           335       52      -283     
     Lines         14981     1852    -13129     
     Branches       1505      223     -1282     
   =============================================
   - Hits           7821      186     -7635     
   + Misses         6535     1653     -4882     
   + Partials        625       13      -612     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.04% <ø> (-59.62%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [312 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r559375785



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,90 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
-        .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+            .filterCompletedInstants().lastInstant();
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+    if (compactType != CompactType.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
     }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
-    HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
-    try {
-      SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
-      Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
-          .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
-          .collect(Collectors.toSet());
-      // exclude files in pending clustering from compaction.
-      fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
-      return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+  public boolean needCompact(CompactType compactType) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = checkCompact(compactType);
+    switch (compactType) {
+      case COMMIT_NUM:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1;
+        break;
+      case TIME_ELAPSED:
+        compactable = parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_OR_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            || parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_AND_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            && parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      default:
+        throw new HoodieCompactionException("Unsupported compact type: " + config.getInlineCompactType());
+    }
 
-    } catch (IOException e) {
-      throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+    if (compactable) {
+      LOG.info(String.format("Scheduling compaction: %s. Delta commits found: %s times, and last compaction time is %s.",

Review comment:
       reasonable, The key point that I want to raise up is: it would be better to describe the trigger condition.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-767261660


   cc @yanghua 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (3639b90) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **increase** coverage by `0.01%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   + Coverage     52.20%   52.21%   +0.01%     
   - Complexity     2659     2662       +3     
   ============================================
     Files           335      335              
     Lines         14981    14981              
     Branches       1505     1506       +1     
   ============================================
   + Hits           7821     7823       +2     
   + Misses         6535     6534       -1     
   + Partials        625      624       -1     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.83% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `54.74% <ø> (-0.03%)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `33.52% <ø> (+0.23%)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `69.65% <ø> (ø)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...e/hudi/common/table/log/HoodieLogFormatWriter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVMb2dGb3JtYXRXcml0ZXIuamF2YQ==) | `77.11% <0.00%> (-1.70%)` | `23.00% <0.00%> (ø%)` | |
   | [...ache/hudi/hadoop/utils/HoodieInputFormatUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL3V0aWxzL0hvb2RpZUlucHV0Rm9ybWF0VXRpbHMuamF2YQ==) | `50.32% <0.00%> (+2.61%)` | `23.00% <0.00%> (+3.00%)` | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r570771434



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
##########
@@ -90,6 +90,20 @@ public static String createNewInstantTime() {
     });
   }
 
+  /**
+   * Returns next instant time that adds milliseconds in the {@link #COMMIT_FORMATTER} format.
+   * Ensures each instant time is atleast 1 second apart since we create instant times at second granularity
+   */
+  public static String createNewInstantTime(long milliseconds) {

Review comment:
       ok




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (0541c81) into [master](https://codecov.io/gh/apache/hudi/commit/6a6b772c4980106efe71e3799da9e6a00746dd4e?el=desc) (6a6b772) will **decrease** coverage by `0.17%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   - Coverage     52.38%   52.20%   -0.18%     
   - Complexity     2632     2659      +27     
   ============================================
     Files           329      335       +6     
     Lines         14808    14981     +173     
     Branches       1484     1505      +21     
   ============================================
   + Hits           7757     7821      +64     
   - Misses         6437     6535      +98     
   - Partials        614      625      +11     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.83% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `54.76% <ø> (-0.37%)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `33.29% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `69.65% <ø> (-0.47%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ache/hudi/common/fs/inline/InMemoryFileSystem.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL2lubGluZS9Jbk1lbW9yeUZpbGVTeXN0ZW0uamF2YQ==) | `79.31% <0.00%> (-10.35%)` | `15.00% <0.00%> (-1.00%)` | |
   | [...di/common/table/timeline/HoodieActiveTimeline.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZUFjdGl2ZVRpbWVsaW5lLmphdmE=) | `69.62% <0.00%> (-4.51%)` | `41.00% <0.00%> (ø%)` | |
   | [...che/hudi/common/table/timeline/HoodieTimeline.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZVRpbWVsaW5lLmphdmE=) | `89.13% <0.00%> (-4.06%)` | `43.00% <0.00%> (ø%)` | |
   | [...g/apache/hudi/common/model/WriteOperationType.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL1dyaXRlT3BlcmF0aW9uVHlwZS5qYXZh) | `53.33% <0.00%> (-1.84%)` | `2.00% <0.00%> (ø%)` | |
   | [...a/org/apache/hudi/common/util/ClusteringUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvQ2x1c3RlcmluZ1V0aWxzLmphdmE=) | `89.06% <0.00%> (-1.42%)` | `18.00% <0.00%> (ø%)` | |
   | [.../java/org/apache/hudi/common/util/StringUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvU3RyaW5nVXRpbHMuamF2YQ==) | `66.66% <0.00%> (ø)` | `14.00% <0.00%> (ø%)` | |
   | [...ies/sources/helpers/DatePartitionPathSelector.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvaGVscGVycy9EYXRlUGFydGl0aW9uUGF0aFNlbGVjdG9yLmphdmE=) | `54.83% <0.00%> (ø)` | `13.00% <0.00%> (?%)` | |
   | [.../apache/hudi/common/model/ClusteringOperation.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0NsdXN0ZXJpbmdPcGVyYXRpb24uamF2YQ==) | `0.00% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | [...g/apache/hudi/common/model/HoodiePayloadProps.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0hvb2RpZVBheWxvYWRQcm9wcy5qYXZh) | `0.00% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | [...e/hudi/common/table/log/HoodieFileSliceReader.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVGaWxlU2xpY2VSZWFkZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | ... and [6 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (0ae6c5a) into [master](https://codecov.io/gh/apache/hudi/commit/6a6b772c4980106efe71e3799da9e6a00746dd4e?el=desc) (6a6b772) will **decrease** coverage by `0.17%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   - Coverage     52.38%   52.20%   -0.18%     
   - Complexity     2632     2659      +27     
   ============================================
     Files           329      335       +6     
     Lines         14808    14981     +173     
     Branches       1484     1505      +21     
   ============================================
   + Hits           7757     7821      +64     
   - Misses         6437     6535      +98     
   - Partials        614      625      +11     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.83% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `54.76% <ø> (-0.37%)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `33.29% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `69.65% <ø> (-0.47%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ache/hudi/common/fs/inline/InMemoryFileSystem.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL2lubGluZS9Jbk1lbW9yeUZpbGVTeXN0ZW0uamF2YQ==) | `79.31% <0.00%> (-10.35%)` | `15.00% <0.00%> (-1.00%)` | |
   | [...di/common/table/timeline/HoodieActiveTimeline.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZUFjdGl2ZVRpbWVsaW5lLmphdmE=) | `69.62% <0.00%> (-4.51%)` | `41.00% <0.00%> (ø%)` | |
   | [...che/hudi/common/table/timeline/HoodieTimeline.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZVRpbWVsaW5lLmphdmE=) | `89.13% <0.00%> (-4.06%)` | `43.00% <0.00%> (ø%)` | |
   | [...g/apache/hudi/common/model/WriteOperationType.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL1dyaXRlT3BlcmF0aW9uVHlwZS5qYXZh) | `53.33% <0.00%> (-1.84%)` | `2.00% <0.00%> (ø%)` | |
   | [...a/org/apache/hudi/common/util/ClusteringUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvQ2x1c3RlcmluZ1V0aWxzLmphdmE=) | `89.06% <0.00%> (-1.42%)` | `18.00% <0.00%> (ø%)` | |
   | [.../java/org/apache/hudi/common/util/StringUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvU3RyaW5nVXRpbHMuamF2YQ==) | `66.66% <0.00%> (ø)` | `14.00% <0.00%> (ø%)` | |
   | [.../hudi/common/model/DefaultHoodieRecordPayload.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0RlZmF1bHRIb29kaWVSZWNvcmRQYXlsb2FkLmphdmE=) | `64.28% <0.00%> (ø)` | `4.00% <0.00%> (?%)` | |
   | [.../apache/hudi/common/model/ClusteringOperation.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0NsdXN0ZXJpbmdPcGVyYXRpb24uamF2YQ==) | `0.00% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | [...e/hudi/common/table/log/HoodieFileSliceReader.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVGaWxlU2xpY2VSZWFkZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | [.../apache/hudi/common/config/SerializableSchema.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2NvbmZpZy9TZXJpYWxpemFibGVTY2hlbWEuamF2YQ==) | `57.89% <0.00%> (ø)` | `6.00% <0.00%> (?%)` | |
   | ... and [6 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (6ff932e) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **decrease** coverage by `42.16%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     52.20%   10.04%   -42.17%     
   + Complexity     2659       48     -2611     
   =============================================
     Files           335       52      -283     
     Lines         14981     1852    -13129     
     Branches       1505      223     -1282     
   =============================================
   - Hits           7821      186     -7635     
   + Misses         6535     1653     -4882     
   + Partials        625       13      -612     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.04% <ø> (-59.62%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [312 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (0750f24) into [master](https://codecov.io/gh/apache/hudi/commit/4d05680038752077ceaebef261b66a5afc761e10?el=desc) (4d05680) will **decrease** coverage by `43.12%`.
   > The diff coverage is `13.04%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     53.53%   10.40%   -43.13%     
   + Complexity     2770       48     -2722     
   =============================================
     Files           348       51      -297     
     Lines         16109     1787    -14322     
     Branches       1643      213     -1430     
   =============================================
   - Hits           8624      186     -8438     
   + Misses         6786     1588     -5198     
   + Partials        699       13      -686     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudispark | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.40% <13.04%> (-59.69%)` | `0.00 <1.00> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `0.00% <0.00%> (-70.55%)` | `0.00 <0.00> (-49.00)` | |
   | [...i/utilities/deltastreamer/HoodieDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllRGVsdGFTdHJlYW1lci5qYXZh) | `0.00% <0.00%> (-69.24%)` | `0.00 <0.00> (-18.00)` | |
   | [...s/deltastreamer/HoodieMultiTableDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllTXVsdGlUYWJsZURlbHRhU3RyZWFtZXIuamF2YQ==) | `0.00% <0.00%> (-78.40%)` | `0.00 <0.00> (-18.00)` | |
   | [...i/utilities/deltastreamer/SourceFormatAdapter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvU291cmNlRm9ybWF0QWRhcHRlci5qYXZh) | `0.00% <0.00%> (-86.49%)` | `0.00 <0.00> (-11.00)` | |
   | [.../hudi/utilities/schema/SparkAvroPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TcGFya0F2cm9Qb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...in/java/org/apache/hudi/utilities/UtilHelpers.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL1V0aWxIZWxwZXJzLmphdmE=) | `35.54% <16.66%> (-28.66%)` | `11.00 <1.00> (-19.00)` | |
   | [.../apache/hudi/utilities/HoodieSnapshotExporter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0hvb2RpZVNuYXBzaG90RXhwb3J0ZXIuamF2YQ==) | `83.47% <100.00%> (-5.12%)` | `28.00 <0.00> (ø)` | |
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [291 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK removed a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK removed a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-730111282


   cc @bvaradar @yanghua 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r559360064



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,90 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
-        .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+            .filterCompletedInstants().lastInstant();
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+    if (compactType != CompactType.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
     }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
-    HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
-    try {
-      SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
-      Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
-          .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
-          .collect(Collectors.toSet());
-      // exclude files in pending clustering from compaction.
-      fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
-      return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+  public boolean needCompact(CompactType compactType) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = checkCompact(compactType);
+    switch (compactType) {
+      case COMMIT_NUM:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1;
+        break;
+      case TIME_ELAPSED:
+        compactable = parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_OR_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            || parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_AND_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            && parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      default:
+        throw new HoodieCompactionException("Unsupported compact type: " + config.getInlineCompactType());
+    }
 
-    } catch (IOException e) {
-      throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+    if (compactable) {
+      LOG.info(String.format("Scheduling compaction: %s. Delta commits found: %s times, and last compaction time is %s.",

Review comment:
       Now we have 4 types of CompactType, and compactable may be true or false.
   which means  it will have 8 situations.  If we add log in to each case, I think a little bit tedious. wdyt?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r570748423



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,97 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());

Review comment:
       yes.just exchange the order. 

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,97 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
+    }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);

Review comment:
       sure




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-758424635


   @wangxianghu 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-730111282


   cc @bvaradar 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vinothchandar merged pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
vinothchandar merged pull request #2260:
URL: https://github.com/apache/hudi/pull/2260


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724






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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (8d37749) into [master](https://codecov.io/gh/apache/hudi/commit/4d05680038752077ceaebef261b66a5afc761e10?el=desc) (4d05680) will **increase** coverage by `0.03%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   + Coverage     53.53%   53.57%   +0.03%     
   - Complexity     2770     2774       +4     
   ============================================
     Files           348      348              
     Lines         16109    16118       +9     
     Branches       1643     1641       -2     
   ============================================
   + Hits           8624     8635      +11     
   + Misses         6786     6785       -1     
   + Partials        699      698       -1     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.50% <ø> (+0.13%)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `55.31% <ø> (+<0.01%)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `32.94% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudispark | `65.73% <0.00%> (+0.14%)` | `0.00 <0.00> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `70.06% <ø> (-0.04%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...in/scala/org/apache/hudi/HoodieStreamingSink.scala](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvSG9vZGllU3RyZWFtaW5nU2luay5zY2FsYQ==) | `23.76% <0.00%> (-0.24%)` | `10.00 <0.00> (ø)` | |
   | [...ache/hudi/common/util/collection/DiskBasedMap.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvY29sbGVjdGlvbi9EaXNrQmFzZWRNYXAuamF2YQ==) | `80.88% <0.00%> (-2.07%)` | `29.00% <0.00%> (+2.00%)` | :arrow_down: |
   | [...i/common/util/collection/ExternalSpillableMap.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvY29sbGVjdGlvbi9FeHRlcm5hbFNwaWxsYWJsZU1hcC5qYXZh) | `71.42% <0.00%> (-1.61%)` | `29.00% <0.00%> (ø%)` | |
   | [.../hudi/common/util/collection/LazyFileIterable.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvY29sbGVjdGlvbi9MYXp5RmlsZUl0ZXJhYmxlLmphdmE=) | `74.41% <0.00%> (-0.59%)` | `2.00% <0.00%> (ø%)` | |
   | [...i/utilities/deltastreamer/HoodieDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllRGVsdGFTdHJlYW1lci5qYXZh) | `68.97% <0.00%> (-0.26%)` | `18.00% <0.00%> (ø%)` | |
   | [...apache/hudi/utilities/deltastreamer/DeltaSync.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvRGVsdGFTeW5jLmphdmE=) | `70.54% <0.00%> (ø)` | `49.00% <0.00%> (ø%)` | |
   | [...s/deltastreamer/HoodieMultiTableDeltaStreamer.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL2RlbHRhc3RyZWFtZXIvSG9vZGllTXVsdGlUYWJsZURlbHRhU3RyZWFtZXIuamF2YQ==) | `78.39% <0.00%> (ø)` | `18.00% <0.00%> (ø%)` | |
   | [...main/scala/org/apache/hudi/DataSourceOptions.scala](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvRGF0YVNvdXJjZU9wdGlvbnMuc2NhbGE=) | `95.00% <0.00%> (+0.08%)` | `0.00% <0.00%> (ø%)` | |
   | [...n/scala/org/apache/hudi/HoodieSparkSqlWriter.scala](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvSG9vZGllU3BhcmtTcWxXcml0ZXIuc2NhbGE=) | `51.50% <0.00%> (+0.55%)` | `0.00% <0.00%> (ø%)` | |
   | [...src/main/java/org/apache/hudi/DataSourceUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvaHVkaS9EYXRhU291cmNlVXRpbHMuamF2YQ==) | `41.90% <0.00%> (+0.95%)` | `20.00% <0.00%> (+1.00%)` | |
   | ... and [2 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (c2a695a) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **increase** coverage by `0.17%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   + Coverage     50.27%   50.44%   +0.17%     
   - Complexity     2990     3063      +73     
   ============================================
     Files           410      420      +10     
     Lines         18406    18904     +498     
     Branches       1885     1938      +53     
   ============================================
   + Hits           9253     9536     +283     
   - Misses         8395     8593     +198     
   - Partials        758      775      +17     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `37.21% <ø> (-0.07%)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `51.45% <0.00%> (-0.44%)` | `0.00 <0.00> (ø)` | |
   | hudiflink | `10.20% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `33.16% <ø> (-0.20%)` | `0.00 <ø> (ø)` | |
   | hudisparkdatasource | `65.90% <ø> (+3.08%)` | `0.00 <ø> (ø)` | |
   | hudisync | `48.61% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `66.49% <ø> (+1.19%)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `69.48% <ø> (-0.18%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...di/common/table/timeline/HoodieActiveTimeline.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZUFjdGl2ZVRpbWVsaW5lLmphdmE=) | `70.64% <0.00%> (-1.33%)` | `42.00 <0.00> (ø)` | |
   | [.../apache/hudi/exception/HoodieCompactException.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvZXhjZXB0aW9uL0hvb2RpZUNvbXBhY3RFeGNlcHRpb24uamF2YQ==) | `0.00% <0.00%> (ø)` | `0.00 <0.00> (?)` | |
   | [...common/table/log/HoodieMergedLogRecordScanner.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVNZXJnZWRMb2dSZWNvcmRTY2FubmVyLmphdmE=) | `87.50% <0.00%> (-4.31%)` | `10.00% <0.00%> (ø%)` | |
   | [...g/apache/hudi/hadoop/HoodieParquetInputFormat.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL0hvb2RpZVBhcnF1ZXRJbnB1dEZvcm1hdC5qYXZh) | `36.26% <0.00%> (-4.15%)` | `11.00% <0.00%> (-1.00%)` | |
   | [.../apache/hudi/common/config/SerializableSchema.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2NvbmZpZy9TZXJpYWxpemFibGVTY2hlbWEuamF2YQ==) | `54.54% <0.00%> (-3.35%)` | `6.00% <0.00%> (ø%)` | |
   | [...che/hudi/common/table/log/HoodieLogFileReader.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVMb2dGaWxlUmVhZGVyLmphdmE=) | `67.85% <0.00%> (-3.22%)` | `22.00% <0.00%> (-1.00%)` | |
   | [...common/table/view/PriorityBasedFileSystemView.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3ZpZXcvUHJpb3JpdHlCYXNlZEZpbGVTeXN0ZW1WaWV3LmphdmE=) | `94.36% <0.00%> (-2.74%)` | `33.00% <0.00%> (ø%)` | |
   | [...org/apache/hudi/hadoop/HoodieHFileInputFormat.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL0hvb2RpZUhGaWxlSW5wdXRGb3JtYXQuamF2YQ==) | `70.45% <0.00%> (-2.10%)` | `10.00% <0.00%> (-1.00%)` | |
   | [...ache/hudi/hadoop/utils/HoodieInputFormatUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL3V0aWxzL0hvb2RpZUlucHV0Rm9ybWF0VXRpbHMuamF2YQ==) | `47.69% <0.00%> (-1.40%)` | `30.00% <0.00%> (+4.00%)` | :arrow_down: |
   | [...ain/java/org/apache/hudi/avro/HoodieAvroUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvYXZyby9Ib29kaWVBdnJvVXRpbHMuamF2YQ==) | `56.09% <0.00%> (-1.19%)` | `37.00% <0.00%> (-4.00%)` | |
   | ... and [44 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (c51004a) into [master](https://codecov.io/gh/apache/hudi/commit/6a6b772c4980106efe71e3799da9e6a00746dd4e?el=desc) (6a6b772) will **decrease** coverage by `0.17%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   - Coverage     52.38%   52.20%   -0.18%     
   - Complexity     2632     2659      +27     
   ============================================
     Files           329      335       +6     
     Lines         14808    14981     +173     
     Branches       1484     1505      +21     
   ============================================
   + Hits           7757     7821      +64     
   - Misses         6437     6535      +98     
   - Partials        614      625      +11     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.83% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `54.76% <ø> (-0.37%)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `33.29% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `69.65% <ø> (-0.47%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ache/hudi/common/fs/inline/InMemoryFileSystem.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL2lubGluZS9Jbk1lbW9yeUZpbGVTeXN0ZW0uamF2YQ==) | `79.31% <0.00%> (-10.35%)` | `15.00% <0.00%> (-1.00%)` | |
   | [...di/common/table/timeline/HoodieActiveTimeline.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZUFjdGl2ZVRpbWVsaW5lLmphdmE=) | `69.62% <0.00%> (-4.51%)` | `41.00% <0.00%> (ø%)` | |
   | [...che/hudi/common/table/timeline/HoodieTimeline.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL0hvb2RpZVRpbWVsaW5lLmphdmE=) | `89.13% <0.00%> (-4.06%)` | `43.00% <0.00%> (ø%)` | |
   | [...g/apache/hudi/common/model/WriteOperationType.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL1dyaXRlT3BlcmF0aW9uVHlwZS5qYXZh) | `53.33% <0.00%> (-1.84%)` | `2.00% <0.00%> (ø%)` | |
   | [...a/org/apache/hudi/common/util/ClusteringUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvQ2x1c3RlcmluZ1V0aWxzLmphdmE=) | `89.06% <0.00%> (-1.42%)` | `18.00% <0.00%> (ø%)` | |
   | [.../java/org/apache/hudi/common/util/StringUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvU3RyaW5nVXRpbHMuamF2YQ==) | `66.66% <0.00%> (ø)` | `14.00% <0.00%> (ø%)` | |
   | [...g/apache/hudi/common/model/HoodiePayloadProps.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0hvb2RpZVBheWxvYWRQcm9wcy5qYXZh) | `0.00% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | [...e/hudi/common/table/log/HoodieFileSliceReader.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVGaWxlU2xpY2VSZWFkZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | `0.00% <0.00%> (?%)` | |
   | [.../hudi/common/model/DefaultHoodieRecordPayload.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL0RlZmF1bHRIb29kaWVSZWNvcmRQYXlsb2FkLmphdmE=) | `64.28% <0.00%> (ø)` | `4.00% <0.00%> (?%)` | |
   | [.../apache/hudi/common/config/SerializableSchema.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2NvbmZpZy9TZXJpYWxpemFibGVTY2hlbWEuamF2YQ==) | `57.89% <0.00%> (ø)` | `6.00% <0.00%> (?%)` | |
   | ... and [6 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (6ff932e) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **decrease** coverage by `42.16%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     52.20%   10.04%   -42.17%     
   + Complexity     2659       48     -2611     
   =============================================
     Files           335       52      -283     
     Lines         14981     1852    -13129     
     Branches       1505      223     -1282     
   =============================================
   - Hits           7821      186     -7635     
   + Misses         6535     1653     -4882     
   + Partials        625       13      -612     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.04% <ø> (-59.62%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [312 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK removed a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK removed a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-747348473


   @vinothchandar 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r532042551



##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
##########
@@ -39,17 +39,18 @@
 
 public class TestInlineCompaction extends CompactionTestBase {
 
-  private HoodieWriteConfig getConfigForInlineCompaction(int maxDeltaCommits) {
+  private HoodieWriteConfig getConfigForInlineCompaction(int maxDeltaCommits, int maxDeltaTime) {
     return getConfigBuilder(false)
         .withCompactionConfig(HoodieCompactionConfig.newBuilder()
-            .withInlineCompaction(true).withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommits).build())
+            .withInlineCompaction(true).withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommits)

Review comment:
       break the second `with` to a new line?

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
##########
@@ -315,7 +315,8 @@ public HoodieActiveTimeline getActiveTimeline() {
    */
   public abstract Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context,
                                                                   String instantTime,
-                                                                  Option<Map<String, String>> extraMetadata);
+                                                                  Option<Map<String, String>> extraMetadata,
+                                                                  String initialTime);

Review comment:
       Missing doc about `initialTime `.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r560755205



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##########
@@ -109,6 +112,8 @@
   private static final String DEFAULT_INLINE_COMPACT = "false";
   private static final String DEFAULT_INCREMENTAL_CLEANER = "true";
   private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "5";
+  private static final String DEFAULT_INLINE_COMPACT_ELAPSED_TIME = String.valueOf(60 * 60);

Review comment:
       ditto

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,112 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
+    }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+  public boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs

Review comment:
       `return` -> `get`

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
##########
@@ -85,32 +88,185 @@ public void testSuccessfulCompaction() throws Exception {
   }
 
   @Test
-  public void testCompactionRetryOnFailure() throws Exception {
+  public void testSuccessfulCompactionBasedOnTime() throws Exception {
+    // Given: make one commit
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(5, 10, CompactionTriggerStrategy.TIME_ELAPSED);
+
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      String instantTime = HoodieActiveTimeline.createNewInstantTime();
+      List<HoodieRecord> records = dataGen.generateInserts(instantTime, 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      runNextDeltaCommits(writeClient, readClient, Arrays.asList(instantTime), records, cfg, true, new ArrayList<>());
+
+      // after 10s, that will trigger compaction
+      Thread.sleep(10000);
+      String finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 100), writeClient, metaClient, cfg, false);
+
+      // Then: ensure the file slices are compacted as per policy
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(3, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+      assertEquals(HoodieTimeline.COMMIT_ACTION, metaClient.getActiveTimeline().lastInstant().get().getAction());
+    }
+  }
+
+  @Test
+  public void testSuccessfulCompactionBasedOnNumOrTime() throws Exception {
+    // Given: make three commits
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(3, 20, CompactionTriggerStrategy.NUM_OR_TIME);
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      List<HoodieRecord> records = dataGen.generateInserts(HoodieActiveTimeline.createNewInstantTime(), 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      List<String> instants = IntStream.range(0, 2).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList());
+      runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
+      // Then: trigger the compaction because reach 3 commits.
+      String finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false);
+
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+      Thread.sleep(20000);

Review comment:
       ditto

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
##########
@@ -85,32 +88,185 @@ public void testSuccessfulCompaction() throws Exception {
   }
 
   @Test
-  public void testCompactionRetryOnFailure() throws Exception {
+  public void testSuccessfulCompactionBasedOnTime() throws Exception {
+    // Given: make one commit
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(5, 10, CompactionTriggerStrategy.TIME_ELAPSED);
+
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      String instantTime = HoodieActiveTimeline.createNewInstantTime();
+      List<HoodieRecord> records = dataGen.generateInserts(instantTime, 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      runNextDeltaCommits(writeClient, readClient, Arrays.asList(instantTime), records, cfg, true, new ArrayList<>());
+
+      // after 10s, that will trigger compaction
+      Thread.sleep(10000);
+      String finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 100), writeClient, metaClient, cfg, false);
+
+      // Then: ensure the file slices are compacted as per policy
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(3, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+      assertEquals(HoodieTimeline.COMMIT_ACTION, metaClient.getActiveTimeline().lastInstant().get().getAction());
+    }
+  }
+
+  @Test
+  public void testSuccessfulCompactionBasedOnNumOrTime() throws Exception {
+    // Given: make three commits
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(3, 20, CompactionTriggerStrategy.NUM_OR_TIME);
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      List<HoodieRecord> records = dataGen.generateInserts(HoodieActiveTimeline.createNewInstantTime(), 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      List<String> instants = IntStream.range(0, 2).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList());
+      runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
+      // Then: trigger the compaction because reach 3 commits.
+      String finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false);
+
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+      Thread.sleep(20000);
+      // 4th commit, that will trigger compaction because reach the time elapsed
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false);
+
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(6, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+    }
+  }
+
+  @Test
+  public void testSuccessfulCompactionBasedOnNumAndTime() throws Exception {
+    // Given: make three commits
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(3, 20, CompactionTriggerStrategy.NUM_AND_TIME);
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      List<HoodieRecord> records = dataGen.generateInserts(HoodieActiveTimeline.createNewInstantTime(), 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      List<String> instants = IntStream.range(0, 3).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList());
+      runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+
+      // Then: ensure no compaction is executedm since there are only 3 delta commits
+      assertEquals(3, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+      Thread.sleep(20000);
+      // 4th commit, that will trigger compaction
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      String finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false);
+
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(5, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+    }
+  }
+
+  @Test
+  public void testCompactionRetryOnFailureBasedOnNumCommits() throws Exception {
+    // Given: two commits, schedule compaction and its failed/in-flight
+    HoodieWriteConfig cfg = getConfigBuilder(false)
+        .withCompactionConfig(HoodieCompactionConfig.newBuilder()
+            .withInlineCompaction(false)
+            .withMaxNumDeltaCommitsBeforeCompaction(1).build())
+        .build();
+    List<String> instants = IntStream.range(0, 2).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList());
+    String instantTime2;
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      List<HoodieRecord> records = dataGen.generateInserts(instants.get(0), 100);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
+      // Schedule compaction instant2, make it in-flight (simulates inline compaction failing)
+      instantTime2 = HoodieActiveTimeline.createNewInstantTime();
+      scheduleCompaction(instantTime2, writeClient, cfg);
+      moveCompactionFromRequestedToInflight(instantTime2, cfg);
+    }
+
+    // When: a third commit happens
+    HoodieWriteConfig inlineCfg = getConfigForInlineCompaction(2, 60, CompactionTriggerStrategy.NUM);
+    String instantTime3 = HoodieActiveTimeline.createNewInstantTime();
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(inlineCfg)) {
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      createNextDeltaCommit(instantTime3, dataGen.generateUpdates(instantTime3, 100), writeClient, metaClient, inlineCfg, false);
+    }
+
+    // Then: 1 delta commit is done, the failed compaction is retried
+    metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+    assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+    assertEquals(instantTime2, metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().firstInstant().get().getTimestamp());
+  }
+
+  @Test
+  public void testCompactionRetryOnFailureBasedOnTime() throws Exception {
     // Given: two commits, schedule compaction and its failed/in-flight
     HoodieWriteConfig cfg = getConfigBuilder(false)
         .withCompactionConfig(HoodieCompactionConfig.newBuilder()
-            .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
+            .withInlineCompaction(false)
+            .withMaxDeltaTimeBeforeCompaction(5)
+            .withInlineCompactionTriggerStrategy(CompactionTriggerStrategy.TIME_ELAPSED).build())
         .build();
-    List<String> instants = CollectionUtils.createImmutableList("000", "001");
+    String instantTime;
+    List<String> instants = IntStream.range(0, 2).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList());
     try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
       List<HoodieRecord> records = dataGen.generateInserts(instants.get(0), 100);
       HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
       runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
-      // Schedule compaction 002, make it in-flight (simulates inline compaction failing)
-      scheduleCompaction("002", writeClient, cfg);
-      moveCompactionFromRequestedToInflight("002", cfg);
+      // Schedule compaction instantTime, make it in-flight (simulates inline compaction failing)
+      Thread.sleep(10000);

Review comment:
       ditto

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##########
@@ -46,6 +47,8 @@
   public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
   // Run a compaction every N delta commits
   public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
+  public static final String INLINE_COMPACT_ELAPSED_TIME_PROP = "hoodie.compact.inline.max.delta.seconds";

Review comment:
       Can we unify the constant name to `INLINE_COMPACT_TIME_DELTA_SECONDS_PROP ` so that we can align with `INLINE_COMPACT_NUM_DELTA_COMMITS_PROP ` and `withMaxDeltaTimeBeforeCompaction `

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,112 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
+    }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+  public boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = getLastDeltaCommitInfo(compactionTriggerStrategy);
+    int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax();
+    int inlineCompactDeltaElapsedTimeMax = config.getInlineCompactDeltaElapsedTimeMax();
+    long elapsedTime;
+    switch (compactionTriggerStrategy) {
+      case NUM:
+        compactable = inlineCompactDeltaCommitMax <= threshold._1;
+        if (compactable) {
+          LOG.info(String.format("The delta commits >= %s, trigger compaction scheduler.", inlineCompactDeltaCommitMax));
+        } else {
+          LOG.info(String.format("Not scheduling compaction because %s delta commits needed since last compaction %s."
+              + "But only %s delta commits found.", inlineCompactDeltaCommitMax, threshold._2, threshold._1));
+        }
+        return compactable;
+      case TIME_ELAPSED:
+        elapsedTime = parsedToSeconds(instantTime) - parsedToSeconds(threshold._2);
+        compactable = inlineCompactDeltaElapsedTimeMax <= elapsedTime;
+        if (compactable) {
+          LOG.info(String.format("The elapsed time >=%ss, trigger compaction scheduler.", inlineCompactDeltaElapsedTimeMax));
+        } else {
+          LOG.info(String.format("Not scheduling compaction because %s elapsed time needed since last compaction %s."
+              + "But only %ss elapsed time found", inlineCompactDeltaElapsedTimeMax, threshold._2, elapsedTime));
+        }
+        return compactable;
+      case NUM_OR_TIME:
+        elapsedTime = parsedToSeconds(instantTime) - parsedToSeconds(threshold._2);
+        compactable = inlineCompactDeltaCommitMax <= threshold._1 || inlineCompactDeltaElapsedTimeMax <= elapsedTime;
+        if (compactable) {
+          LOG.info(String.format("The delta commits >= %s or elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax,
+              inlineCompactDeltaElapsedTimeMax));
+        } else {
+          LOG.info(String.format("Not scheduling compaction because %s delta commits or %ss elapsed time needed since last compaction %s."
+                  + "But only %s delta commits and %ss elapsed time found", inlineCompactDeltaCommitMax, inlineCompactDeltaElapsedTimeMax, threshold._2,
+              threshold._1, elapsedTime));
+        }
+        return compactable;
+      case NUM_AND_TIME:
+        elapsedTime = parsedToSeconds(instantTime) - parsedToSeconds(threshold._2);
+        compactable = inlineCompactDeltaCommitMax <= threshold._1 && inlineCompactDeltaElapsedTimeMax <= elapsedTime;
+        if (compactable) {
+          LOG.info(String.format("The delta commits >= %s and elapsed_time >=%ss, trigger compaction scheduler.", inlineCompactDeltaCommitMax,
+              inlineCompactDeltaElapsedTimeMax));
+        } else {
+          LOG.info(String.format("Not scheduling compaction because %s delta commits and %ss elapsed time needed since last compaction %s."
+                  + "But only %s delta commits and %ss elapsed time found", inlineCompactDeltaCommitMax, inlineCompactDeltaElapsedTimeMax, threshold._2,
+              threshold._1, elapsedTime));
+        }
+        return compactable;
+      default:
+        throw new HoodieCompactionException("Unsupported compact type: " + config.getInlineCompactTriggerStrategy());

Review comment:
       `compact type` -> `compaction trigger strategy`.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,112 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
+    }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+  public boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = getLastDeltaCommitInfo(compactionTriggerStrategy);

Review comment:
       Actually, it's not a threshold, right? it's real value. The below two are threshold values if you want to define.  `threshold` is immutable.

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
##########
@@ -85,32 +88,185 @@ public void testSuccessfulCompaction() throws Exception {
   }
 
   @Test
-  public void testCompactionRetryOnFailure() throws Exception {
+  public void testSuccessfulCompactionBasedOnTime() throws Exception {
+    // Given: make one commit
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(5, 10, CompactionTriggerStrategy.TIME_ELAPSED);
+
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      String instantTime = HoodieActiveTimeline.createNewInstantTime();
+      List<HoodieRecord> records = dataGen.generateInserts(instantTime, 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      runNextDeltaCommits(writeClient, readClient, Arrays.asList(instantTime), records, cfg, true, new ArrayList<>());
+
+      // after 10s, that will trigger compaction
+      Thread.sleep(10000);

Review comment:
       We should avoid use sleep, it will add the CI time. Can we fetch the relevant status with a loop?

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,112 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
+    }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+  public boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = getLastDeltaCommitInfo(compactionTriggerStrategy);
+    int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax();
+    int inlineCompactDeltaElapsedTimeMax = config.getInlineCompactDeltaElapsedTimeMax();
+    long elapsedTime;
+    switch (compactionTriggerStrategy) {
+      case NUM:
+        compactable = inlineCompactDeltaCommitMax <= threshold._1;
+        if (compactable) {
+          LOG.info(String.format("The delta commits >= %s, trigger compaction scheduler.", inlineCompactDeltaCommitMax));
+        } else {
+          LOG.info(String.format("Not scheduling compaction because %s delta commits needed since last compaction %s."

Review comment:
       IMO, we do not need all the `else` statement, right? It's too normal if we do not match the compaction strategy.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,112 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
+    }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+  public boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = getLastDeltaCommitInfo(compactionTriggerStrategy);
+    int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax();
+    int inlineCompactDeltaElapsedTimeMax = config.getInlineCompactDeltaElapsedTimeMax();
+    long elapsedTime;
+    switch (compactionTriggerStrategy) {
+      case NUM:
+        compactable = inlineCompactDeltaCommitMax <= threshold._1;
+        if (compactable) {
+          LOG.info(String.format("The delta commits >= %s, trigger compaction scheduler.", inlineCompactDeltaCommitMax));
+        } else {
+          LOG.info(String.format("Not scheduling compaction because %s delta commits needed since last compaction %s."
+              + "But only %s delta commits found.", inlineCompactDeltaCommitMax, threshold._2, threshold._1));
+        }
+        return compactable;

Review comment:
       Since you have defined a `compactable `, let's use `break` here and return it in the end.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,112 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {

Review comment:
       Do we also need to judge `NUM_OR_TIME `?

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
##########
@@ -85,32 +88,185 @@ public void testSuccessfulCompaction() throws Exception {
   }
 
   @Test
-  public void testCompactionRetryOnFailure() throws Exception {
+  public void testSuccessfulCompactionBasedOnTime() throws Exception {
+    // Given: make one commit
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(5, 10, CompactionTriggerStrategy.TIME_ELAPSED);
+
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      String instantTime = HoodieActiveTimeline.createNewInstantTime();
+      List<HoodieRecord> records = dataGen.generateInserts(instantTime, 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      runNextDeltaCommits(writeClient, readClient, Arrays.asList(instantTime), records, cfg, true, new ArrayList<>());
+
+      // after 10s, that will trigger compaction
+      Thread.sleep(10000);
+      String finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 100), writeClient, metaClient, cfg, false);
+
+      // Then: ensure the file slices are compacted as per policy
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(3, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+      assertEquals(HoodieTimeline.COMMIT_ACTION, metaClient.getActiveTimeline().lastInstant().get().getAction());
+    }
+  }
+
+  @Test
+  public void testSuccessfulCompactionBasedOnNumOrTime() throws Exception {
+    // Given: make three commits
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(3, 20, CompactionTriggerStrategy.NUM_OR_TIME);
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      List<HoodieRecord> records = dataGen.generateInserts(HoodieActiveTimeline.createNewInstantTime(), 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      List<String> instants = IntStream.range(0, 2).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList());
+      runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
+      // Then: trigger the compaction because reach 3 commits.
+      String finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false);
+
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+      Thread.sleep(20000);
+      // 4th commit, that will trigger compaction because reach the time elapsed
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false);
+
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(6, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+    }
+  }
+
+  @Test
+  public void testSuccessfulCompactionBasedOnNumAndTime() throws Exception {
+    // Given: make three commits
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(3, 20, CompactionTriggerStrategy.NUM_AND_TIME);
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      List<HoodieRecord> records = dataGen.generateInserts(HoodieActiveTimeline.createNewInstantTime(), 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      List<String> instants = IntStream.range(0, 3).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList());
+      runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+
+      // Then: ensure no compaction is executedm since there are only 3 delta commits
+      assertEquals(3, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+      Thread.sleep(20000);

Review comment:
       ditto

##########
File path: hudi-common/src/main/java/org/apache/hudi/exception/HoodieCompactException.java
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.exception;
+
+public class HoodieCompactException extends HoodieException {

Review comment:
       Useless exception class?

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
##########
@@ -85,32 +88,185 @@ public void testSuccessfulCompaction() throws Exception {
   }
 
   @Test
-  public void testCompactionRetryOnFailure() throws Exception {
+  public void testSuccessfulCompactionBasedOnTime() throws Exception {
+    // Given: make one commit
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(5, 10, CompactionTriggerStrategy.TIME_ELAPSED);
+
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      String instantTime = HoodieActiveTimeline.createNewInstantTime();
+      List<HoodieRecord> records = dataGen.generateInserts(instantTime, 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      runNextDeltaCommits(writeClient, readClient, Arrays.asList(instantTime), records, cfg, true, new ArrayList<>());
+
+      // after 10s, that will trigger compaction
+      Thread.sleep(10000);
+      String finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 100), writeClient, metaClient, cfg, false);
+
+      // Then: ensure the file slices are compacted as per policy
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(3, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+      assertEquals(HoodieTimeline.COMMIT_ACTION, metaClient.getActiveTimeline().lastInstant().get().getAction());
+    }
+  }
+
+  @Test
+  public void testSuccessfulCompactionBasedOnNumOrTime() throws Exception {
+    // Given: make three commits
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(3, 20, CompactionTriggerStrategy.NUM_OR_TIME);
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      List<HoodieRecord> records = dataGen.generateInserts(HoodieActiveTimeline.createNewInstantTime(), 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      List<String> instants = IntStream.range(0, 2).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList());
+      runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
+      // Then: trigger the compaction because reach 3 commits.
+      String finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false);
+
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+      Thread.sleep(20000);
+      // 4th commit, that will trigger compaction because reach the time elapsed
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      finalInstant = HoodieActiveTimeline.createNewInstantTime();
+      createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 10), writeClient, metaClient, cfg, false);
+
+      metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+      assertEquals(6, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+    }
+  }
+
+  @Test
+  public void testSuccessfulCompactionBasedOnNumAndTime() throws Exception {
+    // Given: make three commits
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(3, 20, CompactionTriggerStrategy.NUM_AND_TIME);
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      List<HoodieRecord> records = dataGen.generateInserts(HoodieActiveTimeline.createNewInstantTime(), 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      List<String> instants = IntStream.range(0, 3).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList());
+      runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
+
+      // Then: ensure no compaction is executedm since there are only 3 delta commits
+      assertEquals(3, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
+      Thread.sleep(20000);

Review comment:
       ditto




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (7d0453e) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **decrease** coverage by `40.58%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2260       +/-   ##
   ============================================
   - Coverage     50.27%   9.68%   -40.59%     
   + Complexity     2990      48     -2942     
   ============================================
     Files           410      53      -357     
     Lines         18406    1930    -16476     
     Branches       1885     230     -1655     
   ============================================
   - Hits           9253     187     -9066     
   + Misses         8395    1730     -6665     
   + Partials        758      13      -745     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.97%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [388 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r553075502



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,23 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction

Review comment:
       Currently, compaction will be triggered once reach one of the conditions.
   `Can users only choose one of them?`
   No, but they can set time or commitsNum to Infinity
   




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (d087f09) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **decrease** coverage by `40.62%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2260       +/-   ##
   ============================================
   - Coverage     50.27%   9.64%   -40.63%     
   + Complexity     2990      48     -2942     
   ============================================
     Files           410      53      -357     
     Lines         18406    1929    -16477     
     Branches       1885     230     -1655     
   ============================================
   - Hits           9253     186     -9067     
   + Misses         8395    1730     -6665     
   + Partials        758      13      -745     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.64% <ø> (-60.02%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [388 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r559695716



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,98 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
-        .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+            .filterCompletedInstants().lastInstant();
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+    if (compactType != CompactType.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
     }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
-    HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
-    try {
-      SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
-      Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
-          .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
-          .collect(Collectors.toSet());
-      // exclude files in pending clustering from compaction.
-      fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
-      return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+  public boolean needCompact(CompactType compactType) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = checkCompact(compactType);
+    int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax();
+    int inlineCompactDeltaElapsedTimeMax = config.getInlineCompactDeltaElapsedTimeMax();
+    switch (compactType) {
+      case COMMIT_NUM:
+        compactable = inlineCompactDeltaCommitMax <= threshold._1;
+        LOG.info(String.format("Trigger compaction when commit_num >=%s", inlineCompactDeltaCommitMax));
+        break;
+      case TIME_ELAPSED:
+        compactable = parseToTimestamp(threshold._2) + inlineCompactDeltaElapsedTimeMax <= parseToTimestamp(instantTime);
+        LOG.info(String.format("Trigger compaction when elapsed_time >=%ss", inlineCompactDeltaElapsedTimeMax));
+        break;
+      case NUM_OR_TIME:
+        compactable = inlineCompactDeltaCommitMax <= threshold._1
+            || parseToTimestamp(threshold._2) + inlineCompactDeltaElapsedTimeMax <= parseToTimestamp(instantTime);
+        LOG.info(String.format("Trigger compaction when commit_num >=%s or elapsed_time >=%ss", inlineCompactDeltaCommitMax,
+                inlineCompactDeltaElapsedTimeMax));
+        break;
+      case NUM_AND_TIME:
+        compactable = inlineCompactDeltaCommitMax <= threshold._1
+            && parseToTimestamp(threshold._2) + inlineCompactDeltaElapsedTimeMax <= parseToTimestamp(instantTime);
+        LOG.info(String.format("Trigger compaction when commit_num >=%s and elapsed_time >=%ss", inlineCompactDeltaCommitMax,
+                inlineCompactDeltaElapsedTimeMax));
+        break;
+      default:
+        throw new HoodieCompactionException("Unsupported compact type: " + config.getInlineCompactType());
+    }
 
-    } catch (IOException e) {
-      throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+    if (compactable) {
+      LOG.info(String.format("Scheduling compaction: %s. Delta commits found: %s times, and last compaction time is %s.",
+              compactType.name(), threshold._1, threshold._2));
+    } else {
+      LOG.info(String.format("Not scheduling compaction as only %s delta commits was found since last compaction %s."
+                      + "Waiting for %s,or %sms elapsed time need since last compaction %s.", threshold._1,
+              threshold._2, config.getInlineCompactDeltaCommitMax(), config.getInlineCompactDeltaElapsedTimeMax(), threshold._2));
     }
+    return compactable;
   }
 
+  public Long parseToTimestamp(String time) {

Review comment:
       `parsedToSeconds` ? sry . Do u have any good name?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (fbd87b4) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **decrease** coverage by `40.58%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2260       +/-   ##
   ============================================
   - Coverage     50.27%   9.68%   -40.59%     
   + Complexity     2990      48     -2942     
   ============================================
     Files           410      53      -357     
     Lines         18406    1930    -16476     
     Branches       1885     230     -1655     
   ============================================
   - Hits           9253     187     -9066     
   + Misses         8395    1730     -6665     
   + Partials        758      13      -745     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.97%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [388 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (7b63729) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **increase** coverage by `0.02%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   + Coverage     52.20%   52.23%   +0.02%     
   - Complexity     2659     2662       +3     
   ============================================
     Files           335      335              
     Lines         14981    14983       +2     
     Branches       1505     1506       +1     
   ============================================
   + Hits           7821     7826       +5     
   + Misses         6535     6534       -1     
   + Partials        625      623       -2     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.83% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `54.76% <ø> (-0.01%)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `33.52% <ø> (+0.23%)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `69.65% <ø> (ø)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...g/apache/hudi/common/model/WriteOperationType.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL21vZGVsL1dyaXRlT3BlcmF0aW9uVHlwZS5qYXZh) | `53.12% <0.00%> (-0.21%)` | `2.00% <0.00%> (ø%)` | |
   | [...ache/hudi/hadoop/utils/HoodieInputFormatUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL3V0aWxzL0hvb2RpZUlucHV0Rm9ybWF0VXRpbHMuamF2YQ==) | `50.32% <0.00%> (+2.61%)` | `23.00% <0.00%> (+3.00%)` | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io commented on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (679310a) into [master](https://codecov.io/gh/apache/hudi/commit/430d4b428e7c5b325c7414a187f9cda158c2758a?el=desc) (430d4b4) will **decrease** coverage by `43.14%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     53.54%   10.39%   -43.15%     
   + Complexity     2770       48     -2722     
   =============================================
     Files           348       50      -298     
     Lines         16109     1779    -14330     
     Branches       1643      211     -1432     
   =============================================
   - Hits           8626      185     -8441     
   + Misses         6785     1581     -5204     
   + Partials        698       13      -685     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudispark | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.39% <ø> (-59.70%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | ... and [324 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r560999788



##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
##########
@@ -85,32 +88,185 @@ public void testSuccessfulCompaction() throws Exception {
   }
 
   @Test
-  public void testCompactionRetryOnFailure() throws Exception {
+  public void testSuccessfulCompactionBasedOnTime() throws Exception {
+    // Given: make one commit
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(5, 10, CompactionTriggerStrategy.TIME_ELAPSED);
+
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+      String instantTime = HoodieActiveTimeline.createNewInstantTime();
+      List<HoodieRecord> records = dataGen.generateInserts(instantTime, 10);
+      HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
+      runNextDeltaCommits(writeClient, readClient, Arrays.asList(instantTime), records, cfg, true, new ArrayList<>());
+
+      // after 10s, that will trigger compaction
+      Thread.sleep(10000);

Review comment:
       good catch




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724






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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r552590234



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,23 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
+    // judge if we need to compact according to num delta commits and time elapsed
+    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction
+                    && parseTs(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseTs(instantTime)) {
       LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
           + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
+          + config.getInlineCompactDeltaCommitMax() + ". Or " + config.getInlineCompactDeltaElapsedTimeMax()
+              + "ms elapsed time need since last compaction " + lastCompactionTs);

Review comment:
       right




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (0541c81) into [master](https://codecov.io/gh/apache/hudi/commit/6a6b772c4980106efe71e3799da9e6a00746dd4e?el=desc) (6a6b772) will **decrease** coverage by `7.54%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   - Coverage     52.38%   44.83%   -7.55%     
   + Complexity     2632     2361     -271     
   ============================================
     Files           329      335       +6     
     Lines         14808    14981     +173     
     Branches       1484     1505      +21     
   ============================================
   - Hits           7757     6717    -1040     
   - Misses         6437     7755    +1318     
   + Partials        614      509     -105     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.83% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `54.76% <ø> (-0.37%)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `33.29% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `10.04% <ø> (-60.08%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [42 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r560226296



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##########
@@ -46,6 +47,8 @@
   public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
   // Run a compaction every N delta commits
   public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
+  public static final String INLINE_COMPACT_ELAPSED_TIME_PROP = "hoodie.compact.inline.max.delta.time";

Review comment:
       edited. 
   anything else  need to improve? 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r552590421



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,23 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
+    // judge if we need to compact according to num delta commits and time elapsed
+    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction
+                    && parseTs(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseTs(instantTime)) {
       LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
           + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
+          + config.getInlineCompactDeltaCommitMax() + ". Or " + config.getInlineCompactDeltaElapsedTimeMax()
+              + "ms elapsed time need since last compaction " + lastCompactionTs);

Review comment:
       ok




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (3639b90) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **decrease** coverage by `42.16%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     52.20%   10.04%   -42.17%     
   + Complexity     2659       48     -2611     
   =============================================
     Files           335       52      -283     
     Lines         14981     1852    -13129     
     Branches       1505      223     -1282     
   =============================================
   - Hits           7821      186     -7635     
   + Misses         6535     1653     -4882     
   + Partials        625       13      -612     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.04% <ø> (-59.62%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [312 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (87a7489) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **increase** coverage by `0.02%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   + Coverage     52.20%   52.22%   +0.02%     
   - Complexity     2659     2660       +1     
   ============================================
     Files           335      335              
     Lines         14981    14981              
     Branches       1505     1505              
   ============================================
   + Hits           7821     7824       +3     
   + Misses         6535     6532       -3     
     Partials        625      625              
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.83% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `54.80% <ø> (+0.03%)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `33.29% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `69.65% <ø> (ø)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ache/hudi/common/fs/inline/InMemoryFileSystem.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL2lubGluZS9Jbk1lbW9yeUZpbGVTeXN0ZW0uamF2YQ==) | `89.65% <0.00%> (+10.34%)` | `16.00% <0.00%> (+1.00%)` | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r553710167



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,32 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean numCommitEnabled = config.getInlineCompactDeltaNumCommitEnabled();
+    boolean timeEnabled = config.getInlineCompactDeltaElapsedEnabled();
+    boolean compactable;
+    if (numCommitEnabled && !timeEnabled) {
+      compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction;
+    } else if (!numCommitEnabled && timeEnabled) {
+      compactable = parseToTimestamp(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseToTimestamp(instantTime);
+    } else {
+      compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction
+          && parseToTimestamp(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseToTimestamp(instantTime);
+    }

Review comment:
       if we call this function. it means we turn `ASYNC_COMPACT_ENABLE_OPT_KEY` on. Do we still need to check first?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r548980709



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -85,4 +88,15 @@ protected HoodieCompactionPlan scheduleCompaction() {
     }
   }
 
+  protected boolean timeCompaction(String instantTime, String initialTime, int deltaCommitsSinceLastCompaction) {
+    if (Long.parseLong(initialTime) + config.getInlineCompactDeltaElapsedTimeMax() > Long.parseLong(instantTime)) {

Review comment:
       revised




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724






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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724






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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-735227950


   @n3nash  @vinothchandar  hi ,can you pls take a look at this pr?


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r561000385



##########
File path: hudi-common/src/main/java/org/apache/hudi/exception/HoodieCompactException.java
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.exception;
+
+public class HoodieCompactException extends HoodieException {

Review comment:
       used in `SparkScheduleCompactionActionExecutor` 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r560258851



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##########
@@ -46,6 +47,8 @@
   public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
   // Run a compaction every N delta commits
   public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
+  public static final String INLINE_COMPACT_ELAPSED_TIME_PROP = "hoodie.compact.inline.max.delta.time";

Review comment:
       thanks for your patient, will do a final check tomorrow.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724






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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r560998926



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,112 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {

Review comment:
       no need. only `TIME_ELAPSED` don't need `deltaCommitsSinceLastCompaction`




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (d087f09) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **decrease** coverage by `40.62%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2260       +/-   ##
   ============================================
   - Coverage     50.27%   9.64%   -40.63%     
   + Complexity     2990      48     -2942     
   ============================================
     Files           410      53      -357     
     Lines         18406    1929    -16477     
     Branches       1885     230     -1655     
   ============================================
   - Hits           9253     186     -9067     
   + Misses         8395    1730     -6665     
   + Partials        758      13      -745     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.64% <ø> (-60.02%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [388 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724






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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r548886752



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -65,10 +66,12 @@ protected HoodieCompactionPlan scheduleCompaction() {
 
     int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
         .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
+    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction
+                    && timeCompaction(instantTime, initialTime, deltaCommitsSinceLastCompaction)) {

Review comment:
       yes. So two conditions in `if` means they won't trigger compact.
   Return true means imcompactable.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r559364846



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,90 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
-        .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+            .filterCompletedInstants().lastInstant();
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+    if (compactType != CompactType.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
     }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
-    HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
-    try {
-      SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
-      Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
-          .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
-          .collect(Collectors.toSet());
-      // exclude files in pending clustering from compaction.
-      fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
-      return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+  public boolean needCompact(CompactType compactType) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = checkCompact(compactType);
+    switch (compactType) {
+      case COMMIT_NUM:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1;
+        break;
+      case TIME_ELAPSED:
+        compactable = parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_OR_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            || parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_AND_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            && parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      default:
+        throw new HoodieCompactionException("Unsupported compact type: " + config.getInlineCompactType());
+    }
 
-    } catch (IOException e) {
-      throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+    if (compactable) {
+      LOG.info(String.format("Scheduling compaction: %s. Delta commits found: %s times, and last compaction time is %s.",

Review comment:
       so I discuss with wangxianghu , just give a summative statement.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] wangxianghu commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
wangxianghu commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r548791159



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -129,6 +130,7 @@ public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig
     this.metrics = new HoodieMetrics(config, config.getTableName());
     this.rollbackPending = rollbackPending;
     this.index = createIndex(writeConfig);
+    this.initialTime = HoodieActiveTimeline.createNewInstantTime();

Review comment:
       How about mark this `initialTime` as lastCompaction time, then we can gei it from timeline. 
   then, when we get a new commit, we can check the interval between these two timestamps to decide whether execute compact or not.
   in this way:
   1. there is no need to update it additionally
   2. it is more preciseness.(Time elapsed since the last compact)




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] wangxianghu commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
wangxianghu commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r548796771



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -65,10 +66,12 @@ protected HoodieCompactionPlan scheduleCompaction() {
 
     int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
         .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
+    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction
+                    && timeCompaction(instantTime, initialTime, deltaCommitsSinceLastCompaction)) {

Review comment:
       how about triggering a compaction when any one of the conditions is met?

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##########
@@ -109,6 +110,7 @@
   private static final String DEFAULT_INLINE_COMPACT = "false";
   private static final String DEFAULT_INCREMENTAL_CLEANER = "true";
   private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "5";
+  private static final String DEFAULT_INLINE_COMPACT_ELAPSED_TIME = String.valueOf(60 * 60 * 24);

Review comment:
       Do you mean one day?  might be too long, who about an hour

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -85,4 +88,15 @@ protected HoodieCompactionPlan scheduleCompaction() {
     }
   }
 
+  protected boolean timeCompaction(String instantTime, String initialTime, int deltaCommitsSinceLastCompaction) {
+    if (Long.parseLong(initialTime) + config.getInlineCompactDeltaElapsedTimeMax() > Long.parseLong(instantTime)) {

Review comment:
       The timestamp is of `yyyyMMddHHmmss` format, we can not simply sum them by `+`

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -129,6 +130,7 @@ public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig
     this.metrics = new HoodieMetrics(config, config.getTableName());
     this.rollbackPending = rollbackPending;
     this.index = createIndex(writeConfig);
+    this.initialTime = HoodieActiveTimeline.createNewInstantTime();

Review comment:
       How about mark this `initialTime` as lastCompaction time, then we can gei it from timeline. and when we get a new commit, we can check the interval between these two timestamps to decide whether execute compact or not.
   in this way:
   1. there is no need to update it additionally
   2. it is more preciseness.(Time elapsed since the last compact)




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r548950189



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##########
@@ -129,6 +130,7 @@ public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig
     this.metrics = new HoodieMetrics(config, config.getTableName());
     this.rollbackPending = rollbackPending;
     this.index = createIndex(writeConfig);
+    this.initialTime = HoodieActiveTimeline.createNewInstantTime();

Review comment:
       yes. right




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] wangxianghu commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
wangxianghu commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r560098640



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##########
@@ -46,6 +47,8 @@
   public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
   // Run a compaction every N delta commits
   public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
+  public static final String INLINE_COMPACT_ELAPSED_TIME_PROP = "hoodie.compact.inline.max.delta.time";

Review comment:
       how about renaming it to `hoodie.compact.inline.max.delta.seconds`, it seems more readable cc @yanghua 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r570746552



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,97 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {

Review comment:
       yes. I think so. wdyt? @wangxianghu 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r559506139



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactType.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.table.action.compact;
+
+public enum  CompactType {
+    COMMIT_NUM, TIME_ELAPSED, NUM_AND_TIME, NUM_OR_TIME

Review comment:
       `COMMIT_NUM` and `NUM ` do not keep consistent. What about `NUM` or `COMMITS`?
   
   IMO, `CompactType` may make users confused. what about `CompactionTriggerStrategy` Or `CompactionScheduleStrategy`

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,98 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
-        .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+            .filterCompletedInstants().lastInstant();

Review comment:
       Revert this change, please.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,98 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {

Review comment:
       IMO, `getLastDeltaCommitInfo` sounds better? Correct me, if it's not good for you.

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -468,7 +477,7 @@ public String getClusteringExecutionStrategyClass() {
   public long getClusteringMaxBytesInGroup() {
     return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_MAX_BYTES_PER_GROUP));
   }
-  
+

Review comment:
       Still exists?

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -484,7 +493,7 @@ public long getClusteringTargetFileMaxBytes() {
   public int getTargetPartitionsForClustering() {
     return Integer.parseInt(props.getProperty(HoodieClusteringConfig.CLUSTERING_TARGET_PARTITIONS));
   }
-  
+

Review comment:
       ditto

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,98 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
-        .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+            .filterCompletedInstants().lastInstant();
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+    if (compactType != CompactType.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
     }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
-    HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
-    try {
-      SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
-      Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
-          .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
-          .collect(Collectors.toSet());
-      // exclude files in pending clustering from compaction.
-      fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
-      return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+  public boolean needCompact(CompactType compactType) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = checkCompact(compactType);
+    int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax();
+    int inlineCompactDeltaElapsedTimeMax = config.getInlineCompactDeltaElapsedTimeMax();
+    switch (compactType) {
+      case COMMIT_NUM:
+        compactable = inlineCompactDeltaCommitMax <= threshold._1;
+        LOG.info(String.format("Trigger compaction when commit_num >=%s", inlineCompactDeltaCommitMax));
+        break;
+      case TIME_ELAPSED:
+        compactable = parseToTimestamp(threshold._2) + inlineCompactDeltaElapsedTimeMax <= parseToTimestamp(instantTime);
+        LOG.info(String.format("Trigger compaction when elapsed_time >=%ss", inlineCompactDeltaElapsedTimeMax));
+        break;
+      case NUM_OR_TIME:
+        compactable = inlineCompactDeltaCommitMax <= threshold._1
+            || parseToTimestamp(threshold._2) + inlineCompactDeltaElapsedTimeMax <= parseToTimestamp(instantTime);
+        LOG.info(String.format("Trigger compaction when commit_num >=%s or elapsed_time >=%ss", inlineCompactDeltaCommitMax,
+                inlineCompactDeltaElapsedTimeMax));
+        break;
+      case NUM_AND_TIME:
+        compactable = inlineCompactDeltaCommitMax <= threshold._1
+            && parseToTimestamp(threshold._2) + inlineCompactDeltaElapsedTimeMax <= parseToTimestamp(instantTime);
+        LOG.info(String.format("Trigger compaction when commit_num >=%s and elapsed_time >=%ss", inlineCompactDeltaCommitMax,
+                inlineCompactDeltaElapsedTimeMax));
+        break;
+      default:
+        throw new HoodieCompactionException("Unsupported compact type: " + config.getInlineCompactType());
+    }
 
-    } catch (IOException e) {
-      throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+    if (compactable) {
+      LOG.info(String.format("Scheduling compaction: %s. Delta commits found: %s times, and last compaction time is %s.",
+              compactType.name(), threshold._1, threshold._2));
+    } else {
+      LOG.info(String.format("Not scheduling compaction as only %s delta commits was found since last compaction %s."
+                      + "Waiting for %s,or %sms elapsed time need since last compaction %s.", threshold._1,
+              threshold._2, config.getInlineCompactDeltaCommitMax(), config.getInlineCompactDeltaElapsedTimeMax(), threshold._2));
     }
+    return compactable;
   }
 
+  public Long parseToTimestamp(String time) {

Review comment:
       `Timestamp` is not clear, it means `second` or `mills` or something else?

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,98 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
-        .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+            .filterCompletedInstants().lastInstant();
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+    if (compactType != CompactType.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
     }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
-    HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
-    try {
-      SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
-      Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
-          .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
-          .collect(Collectors.toSet());
-      // exclude files in pending clustering from compaction.
-      fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
-      return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+  public boolean needCompact(CompactType compactType) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = checkCompact(compactType);
+    int inlineCompactDeltaCommitMax = config.getInlineCompactDeltaCommitMax();
+    int inlineCompactDeltaElapsedTimeMax = config.getInlineCompactDeltaElapsedTimeMax();
+    switch (compactType) {
+      case COMMIT_NUM:
+        compactable = inlineCompactDeltaCommitMax <= threshold._1;
+        LOG.info(String.format("Trigger compaction when commit_num >=%s", inlineCompactDeltaCommitMax));

Review comment:
       `The commit number is larger than xxx, trigger compaction scheduler.` sounds better?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r548887898



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##########
@@ -109,6 +110,7 @@
   private static final String DEFAULT_INLINE_COMPACT = "false";
   private static final String DEFAULT_INCREMENTAL_CLEANER = "true";
   private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "5";
+  private static final String DEFAULT_INLINE_COMPACT_ELAPSED_TIME = String.valueOf(60 * 60 * 24);

Review comment:
       ok




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (7b63729) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **decrease** coverage by `42.16%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     52.20%   10.04%   -42.17%     
   + Complexity     2659       48     -2611     
   =============================================
     Files           335       52      -283     
     Lines         14981     1852    -13129     
     Branches       1505      223     -1282     
   =============================================
   - Hits           7821      186     -7635     
   + Misses         6535     1653     -4882     
   + Partials        625       13      -612     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.04% <ø> (-59.62%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [312 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK closed pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK closed pull request #2260:
URL: https://github.com/apache/hudi/pull/2260


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r549539352



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -90,4 +99,13 @@ protected HoodieCompactionPlan scheduleCompaction() {
     }
   }
 
+  public Long parseTs(String time) {
+    Long timestamp = null;
+    try {
+      timestamp = HoodieActiveTimeline.COMMIT_FORMATTER.parse(time).getTime() / 1000;
+    } catch (ParseException e) {
+      e.printStackTrace();

Review comment:
       ok




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r570746552



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,97 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {

Review comment:
       yes. I think so. wdyt? @wangxianghu 

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,97 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());

Review comment:
       yes.just exchange the order. 

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,97 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
+    }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);

Review comment:
       sure

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
##########
@@ -90,6 +90,20 @@ public static String createNewInstantTime() {
     });
   }
 
+  /**
+   * Returns next instant time that adds milliseconds in the {@link #COMMIT_FORMATTER} format.
+   * Ensures each instant time is atleast 1 second apart since we create instant times at second granularity
+   */
+  public static String createNewInstantTime(long milliseconds) {

Review comment:
       ok




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK removed a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK removed a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-758424635


   @wangxianghu 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-762017968


   @yanghua ready to merge


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (8d37749) into [master](https://codecov.io/gh/apache/hudi/commit/4d05680038752077ceaebef261b66a5afc761e10?el=desc) (4d05680) will **decrease** coverage by `6.53%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   - Coverage     53.53%   46.99%   -6.54%     
   + Complexity     2770     2495     -275     
   ============================================
     Files           348      348              
     Lines         16109    16118       +9     
     Branches       1643     1641       -2     
   ============================================
   - Hits           8624     7575    -1049     
   - Misses         6786     7958    +1172     
   + Partials        699      585     -114     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.50% <ø> (+0.13%)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `55.31% <ø> (+<0.01%)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `32.94% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudispark | `65.73% <0.00%> (+0.14%)` | `0.00 <0.00> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `10.41% <ø> (-59.69%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...in/scala/org/apache/hudi/HoodieStreamingSink.scala](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvSG9vZGllU3RyZWFtaW5nU2luay5zY2FsYQ==) | `23.76% <0.00%> (-0.24%)` | `10.00 <0.00> (ø)` | |
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | ... and [36 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724






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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vinothchandar commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r569733140



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##########
@@ -46,6 +47,8 @@
   public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
   // Run a compaction every N delta commits
   public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
+  public static final String INLINE_COMPACT_TIME_DELTA_SECONDS_PROP = "hoodie.compact.inline.max.delta.seconds";

Review comment:
       please add comments/java docs explaining what this controls 

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionTriggerStrategy.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.table.action.compact;
+
+public enum CompactionTriggerStrategy {
+    NUM, TIME_ELAPSED, NUM_AND_TIME, NUM_OR_TIME

Review comment:
       rename to `NUM_COMMITS` and add a line of description for each?

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,97 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();

Review comment:
       I understand this is how it was. but overloading `lastCompactionTs` with the first delta commit and reusing this again is hard to grok. Can we atleast rename `lastCompactionTs` -> `latestInstantTs` or something more generic

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,97 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());

Review comment:
       IIUC this block is just moved, no changes to code here within the if block?

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,97 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
+    }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);

Review comment:
       can we use `Pair` instead of Tuple?

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
##########
@@ -90,6 +90,20 @@ public static String createNewInstantTime() {
     });
   }
 
+  /**
+   * Returns next instant time that adds milliseconds in the {@link #COMMIT_FORMATTER} format.
+   * Ensures each instant time is atleast 1 second apart since we create instant times at second granularity
+   */
+  public static String createNewInstantTime(long milliseconds) {

Review comment:
       can this call the method above or otherwise and reduce the code duplication?

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,97 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactTriggerStrategy());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> getLastDeltaCommitInfo(CompactionTriggerStrategy compactionTriggerStrategy) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
+    if (compactionTriggerStrategy != CompactionTriggerStrategy.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {

Review comment:
       can we always compute `deltaCommitsSinceLastCompaction` regardless of strategy. it should be a cheap in-memory operation. then we can merge these two blocks back together




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r553710167



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,32 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean numCommitEnabled = config.getInlineCompactDeltaNumCommitEnabled();
+    boolean timeEnabled = config.getInlineCompactDeltaElapsedEnabled();
+    boolean compactable;
+    if (numCommitEnabled && !timeEnabled) {
+      compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction;
+    } else if (!numCommitEnabled && timeEnabled) {
+      compactable = parseToTimestamp(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseToTimestamp(instantTime);
+    } else {
+      compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction
+          && parseToTimestamp(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseToTimestamp(instantTime);
+    }

Review comment:
       if we call this function. it means we turn `ASYNC_COMPACT_ENABLE_OPT_KEY` on. Do we still need to check first?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] wangxianghu commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
wangxianghu commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r556391080



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,34 +63,64 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = getCompactType(deltaCommitsSinceLastCompaction, lastCompactionTs);

Review comment:
       how about extract all these logic to one method `needCompact(Table table, CompactType compactType )`, and init proper vars when need.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (ddf8fca) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **decrease** coverage by `40.22%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     50.27%   10.04%   -40.23%     
   + Complexity     2990       48     -2942     
   =============================================
     Files           410       52      -358     
     Lines         18406     1852    -16554     
     Branches       1885      223     -1662     
   =============================================
   - Hits           9253      186     -9067     
   + Misses         8395     1653     -6742     
   + Partials        758       13      -745     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.04% <ø> (-59.62%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [387 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-767261660


   cc @yanghua 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (48f7392) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **decrease** coverage by `40.58%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2260       +/-   ##
   ============================================
   - Coverage     50.27%   9.68%   -40.59%     
   + Complexity     2990      48     -2942     
   ============================================
     Files           410      53      -357     
     Lines         18406    1930    -16476     
     Branches       1885     230     -1655     
   ============================================
   - Hits           9253     187     -9066     
   + Misses         8395    1730     -6665     
   + Partials        758      13      -745     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.97%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [388 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (eb7f81b) into [master](https://codecov.io/gh/apache/hudi/commit/6a6b772c4980106efe71e3799da9e6a00746dd4e?el=desc) (6a6b772) will **decrease** coverage by `41.97%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     52.38%   10.40%   -41.98%     
   + Complexity     2632       48     -2584     
   =============================================
     Files           329       51      -278     
     Lines         14808     1787    -13021     
     Branches       1484      213     -1271     
   =============================================
   - Hits           7757      186     -7571     
   + Misses         6437     1588     -4849     
   + Partials        614       13      -601     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.40% <ø> (-59.71%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [306 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-735232969


   @wangxianghu Can you review this PR?


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (b34c416) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **increase** coverage by `19.24%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   + Coverage     50.27%   69.51%   +19.24%     
   + Complexity     2990      357     -2633     
   =============================================
     Files           410       53      -357     
     Lines         18406     1929    -16477     
     Branches       1885      229     -1656     
   =============================================
   - Hits           9253     1341     -7912     
   + Misses         8395      456     -7939     
   + Partials        758      132      -626     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `69.51% <ø> (-0.14%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...common/table/view/FileSystemViewStorageConfig.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3ZpZXcvRmlsZVN5c3RlbVZpZXdTdG9yYWdlQ29uZmlnLmphdmE=) | | | |
   | [...di-cli/src/main/java/org/apache/hudi/cli/Main.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL01haW4uamF2YQ==) | | | |
   | [...java/org/apache/hudi/common/util/ParquetUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvUGFycXVldFV0aWxzLmphdmE=) | | | |
   | [...rg/apache/hudi/common/fs/NoOpConsistencyGuard.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL05vT3BDb25zaXN0ZW5jeUd1YXJkLmphdmE=) | | | |
   | [...e/hudi/common/table/log/HoodieLogFormatWriter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVMb2dGb3JtYXRXcml0ZXIuamF2YQ==) | | | |
   | [...spark3/internal/HoodieDataSourceInternalTable.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3BhcmszL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL3NwYXJrMy9pbnRlcm5hbC9Ib29kaWVEYXRhU291cmNlSW50ZXJuYWxUYWJsZS5qYXZh) | | | |
   | [...sioning/clean/CleanMetadataV1MigrationHandler.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL3ZlcnNpb25pbmcvY2xlYW4vQ2xlYW5NZXRhZGF0YVYxTWlncmF0aW9uSGFuZGxlci5qYXZh) | | | |
   | [.../org/apache/hudi/hive/NonPartitionedExtractor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTm9uUGFydGl0aW9uZWRFeHRyYWN0b3IuamF2YQ==) | | | |
   | [.../main/scala/org/apache/hudi/HoodieSparkUtils.scala](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9odWRpL0hvb2RpZVNwYXJrVXRpbHMuc2NhbGE=) | | | |
   | [...g/apache/hudi/hadoop/HoodieParquetInputFormat.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL0hvb2RpZVBhcnF1ZXRJbnB1dEZvcm1hdC5qYXZh) | | | |
   | ... and [351 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-762018787


   > @yanghua ready to merge
   
   thanks for your reminder, will review it again~


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (6dac804) into [master](https://codecov.io/gh/apache/hudi/commit/4d05680038752077ceaebef261b66a5afc761e10?el=desc) (4d05680) will **decrease** coverage by `43.13%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     53.53%   10.39%   -43.14%     
   + Complexity     2770       48     -2722     
   =============================================
     Files           348       50      -298     
     Lines         16109     1779    -14330     
     Branches       1643      211     -1432     
   =============================================
   - Hits           8624      185     -8439     
   + Misses         6786     1581     -5205     
   + Partials        699       13      -686     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudispark | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.39% <ø> (-59.70%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | ... and [324 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (e1bcc12) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **decrease** coverage by `42.16%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     52.20%   10.04%   -42.17%     
   + Complexity     2659       48     -2611     
   =============================================
     Files           335       52      -283     
     Lines         14981     1852    -13129     
     Branches       1505      223     -1282     
   =============================================
   - Hits           7821      186     -7635     
   + Misses         6535     1653     -4882     
   + Partials        625       13      -612     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.04% <ø> (-59.62%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [312 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r553275932



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,23 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction

Review comment:
       IMO, provicing an option and let user choose a compaction strategy sounds more reasonable. wdyt?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-730111282


   cc @bvaradar @yanghua 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (87a7489) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **decrease** coverage by `42.16%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   - Coverage     52.20%   10.04%   -42.17%     
   + Complexity     2659       48     -2611     
   =============================================
     Files           335       52      -283     
     Lines         14981     1852    -13129     
     Branches       1505      223     -1282     
   =============================================
   - Hits           7821      186     -7635     
   + Misses         6535     1653     -4882     
   + Partials        625       13      -612     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `10.04% <ø> (-59.62%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [312 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r559690708



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,98 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {

Review comment:
       ok, better than mine




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (d720b4e) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **decrease** coverage by `40.58%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2260       +/-   ##
   ============================================
   - Coverage     50.27%   9.68%   -40.59%     
   + Complexity     2990      48     -2942     
   ============================================
     Files           410      53      -357     
     Lines         18406    1930    -16476     
     Branches       1885     230     -1655     
   ============================================
   - Hits           9253     187     -9066     
   + Misses         8395    1730     -6665     
   + Partials        758      13      -745     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.97%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [388 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r552547254



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,23 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
+    // judge if we need to compact according to num delta commits and time elapsed
+    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction
+                    && parseTs(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseTs(instantTime)) {

Review comment:
       Can we define a variable to make the code more readable?

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,23 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
+    // judge if we need to compact according to num delta commits and time elapsed
+    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction
+                    && parseTs(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseTs(instantTime)) {
       LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
           + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
+          + config.getInlineCompactDeltaCommitMax() + ". Or " + config.getInlineCompactDeltaElapsedTimeMax()
+              + "ms elapsed time need since last compaction " + lastCompactionTs);

Review comment:
       IMO, we can use `String.format(...)` to make the log message more readable, right?

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
##########
@@ -85,32 +86,94 @@ public void testSuccessfulCompaction() throws Exception {
   }
 
   @Test
-  public void testCompactionRetryOnFailure() throws Exception {
+  public void testSuccessfulCompactionForTime() throws Exception {

Review comment:
       ditto

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
##########
@@ -39,21 +38,23 @@
 
 public class TestInlineCompaction extends CompactionTestBase {
 
-  private HoodieWriteConfig getConfigForInlineCompaction(int maxDeltaCommits) {
+  private HoodieWriteConfig getConfigForInlineCompaction(int maxDeltaCommits, int maxDeltaTime) {
     return getConfigBuilder(false)
         .withCompactionConfig(HoodieCompactionConfig.newBuilder()
-            .withInlineCompaction(true).withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommits).build())
+            .withInlineCompaction(true).withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommits)

Review comment:
       Can we break the second `withXXX ` into a new line?

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
##########
@@ -62,9 +63,9 @@ public void testCompactionIsNotScheduledEarly() throws Exception {
   }
 
   @Test
-  public void testSuccessfulCompaction() throws Exception {
+  public void testSuccessfulCompactionForNumCommits() throws Exception {

Review comment:
       Replacing `For` to `BasedOn` or `Via` sounds better?

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -90,4 +99,13 @@ protected HoodieCompactionPlan scheduleCompaction() {
     }
   }
 
+  public Long parseTs(String time) {

Review comment:
       Please give it a better name?

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
##########
@@ -85,32 +86,94 @@ public void testSuccessfulCompaction() throws Exception {
   }
 
   @Test
-  public void testCompactionRetryOnFailure() throws Exception {
+  public void testSuccessfulCompactionForTime() throws Exception {
+    // Given: make one commit
+    HoodieWriteConfig cfg = getConfigForInlineCompaction(5,10);

Review comment:
       whitespace between `5,10`




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r552706881



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,23 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction

Review comment:
       Want to make sure: if we need to match the two conditions at the same time? Or they are two different choices? Can users only choose one of them?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r559350017



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.config;
 
+import org.apache.hadoop.hbase.io.compress.Compression;

Review comment:
       Please revert this change.

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -468,7 +477,7 @@ public String getClusteringExecutionStrategyClass() {
   public long getClusteringMaxBytesInGroup() {
     return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_MAX_BYTES_PER_GROUP));
   }
-  

Review comment:
       please revert this change.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,90 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
-        .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+            .filterCompletedInstants().lastInstant();
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+    if (compactType != CompactType.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
     }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
-    HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
-    try {
-      SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
-      Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
-          .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
-          .collect(Collectors.toSet());
-      // exclude files in pending clustering from compaction.
-      fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
-      return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+  public boolean needCompact(CompactType compactType) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = checkCompact(compactType);
+    switch (compactType) {
+      case COMMIT_NUM:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1;
+        break;
+      case TIME_ELAPSED:
+        compactable = parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_OR_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            || parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_AND_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            && parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      default:
+        throw new HoodieCompactionException("Unsupported compact type: " + config.getInlineCompactType());
+    }
 
-    } catch (IOException e) {
-      throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+    if (compactable) {
+      LOG.info(String.format("Scheduling compaction: %s. Delta commits found: %s times, and last compaction time is %s.",
+              compactType.name(), threshold._1, threshold._2));
+    } else {
+      LOG.info(String.format("Not scheduling compaction as only %s delta commits was found since last compaction %s."

Review comment:
       We may not tell why does not trigger compaction. It means nothing happened, right.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,32 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean numCommitEnabled = config.getInlineCompactDeltaNumCommitEnabled();
+    boolean timeEnabled = config.getInlineCompactDeltaElapsedEnabled();
+    boolean compactable;
+    if (numCommitEnabled && !timeEnabled) {
+      compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction;
+    } else if (!numCommitEnabled && timeEnabled) {
+      compactable = parseToTimestamp(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseToTimestamp(instantTime);
+    } else {
+      compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction
+          && parseToTimestamp(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseToTimestamp(instantTime);
+    }

Review comment:
       > if we call this function. it means we turn `ASYNC_COMPACT_ENABLE_OPT_KEY` on. Do we still need to check first?
   
   The switch of `ASYNC_COMPACT_ENABLE_OPT_KEY ` must be the pre-condition. 

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,90 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
-        .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+            .filterCompletedInstants().lastInstant();
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+    if (compactType != CompactType.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
     }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
-    HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
-    try {
-      SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
-      Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
-          .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
-          .collect(Collectors.toSet());
-      // exclude files in pending clustering from compaction.
-      fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
-      return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+  public boolean needCompact(CompactType compactType) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = checkCompact(compactType);
+    switch (compactType) {
+      case COMMIT_NUM:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1;
+        break;
+      case TIME_ELAPSED:
+        compactable = parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_OR_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            || parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_AND_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            && parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      default:
+        throw new HoodieCompactionException("Unsupported compact type: " + config.getInlineCompactType());
+    }
 
-    } catch (IOException e) {
-      throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+    if (compactable) {
+      LOG.info(String.format("Scheduling compaction: %s. Delta commits found: %s times, and last compaction time is %s.",

Review comment:
       We need to describe why it caused compaction, not only some runtime information. e.g. add the log into the case statement?

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
##########
@@ -484,7 +493,7 @@ public long getClusteringTargetFileMaxBytes() {
   public int getTargetPartitionsForClustering() {
     return Integer.parseInt(props.getProperty(HoodieClusteringConfig.CLUSTERING_TARGET_PARTITIONS));
   }
-  

Review comment:
       ditto




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r553411690



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,23 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction

Review comment:
       yes. added!




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (96e596a) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **decrease** coverage by `40.58%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2260       +/-   ##
   ============================================
   - Coverage     50.27%   9.68%   -40.59%     
   + Complexity     2990      48     -2942     
   ============================================
     Files           410      53      -357     
     Lines         18406    1930    -16476     
     Branches       1885     230     -1655     
   ============================================
   - Hits           9253     187     -9066     
   + Misses         8395    1730     -6665     
   + Partials        758      13      -745     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.97%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [388 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] wangxianghu commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
wangxianghu commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r553699091



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -60,17 +63,32 @@ protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
         .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
+      deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean numCommitEnabled = config.getInlineCompactDeltaNumCommitEnabled();
+    boolean timeEnabled = config.getInlineCompactDeltaElapsedEnabled();
+    boolean compactable;
+    if (numCommitEnabled && !timeEnabled) {
+      compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction;
+    } else if (!numCommitEnabled && timeEnabled) {
+      compactable = parseToTimestamp(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseToTimestamp(instantTime);
+    } else {
+      compactable = config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction
+          && parseToTimestamp(lastCompactionTs) + config.getInlineCompactDeltaElapsedTimeMax() > parseToTimestamp(instantTime);
+    }

Review comment:
       Since we have added flags for num style compaction and time elapsed style compaction. maybe we should check the flags first, to make sure at least one of them is enabled. if not, make compact with commits num as default(with a warn log).
   
   besides, we got 4 conditions here:
   1. compact with commit num only;
   2. compact with time elapsed only;
   3. compact when both commit num and time elapsed meet requirements;
   4. compact when one of them is met
   
   WDYT @Karl-WangSK  cc @yanghua 




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (e1bcc12) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **decrease** coverage by `0.01%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   - Coverage     52.20%   52.19%   -0.02%     
     Complexity     2659     2659              
   ============================================
     Files           335      335              
     Lines         14981    14981              
     Branches       1505     1505              
   ============================================
   - Hits           7821     7819       -2     
   - Misses         6535     6536       +1     
   - Partials        625      626       +1     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.83% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `54.74% <ø> (-0.03%)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `33.29% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `69.65% <ø> (ø)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...e/hudi/common/table/log/HoodieLogFormatWriter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVMb2dGb3JtYXRXcml0ZXIuamF2YQ==) | `77.11% <0.00%> (-1.70%)` | `23.00% <0.00%> (ø%)` | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-747348473


   @vinothchandar 


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yanghua commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
yanghua commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r560200475



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
##########
@@ -46,6 +47,8 @@
   public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
   // Run a compaction every N delta commits
   public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
+  public static final String INLINE_COMPACT_ELAPSED_TIME_PROP = "hoodie.compact.inline.max.delta.time";

Review comment:
       sounds good~




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] Karl-WangSK commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
Karl-WangSK commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r559443976



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -58,36 +62,90 @@ public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
   @Override
   protected HoodieCompactionPlan scheduleCompaction() {
     LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    // judge if we need to compact according to num delta commits and time elapsed
+    boolean compactable = needCompact(config.getInlineCompactType());
+    if (compactable) {
+      LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      try {
+        SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+        Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
+            .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+            .collect(Collectors.toSet());
+        // exclude files in pending clustering from compaction.
+        fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
+        return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+      } catch (IOException e) {
+        throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+      }
+    }
+
+    return new HoodieCompactionPlan();
+  }
+
+  public Tuple2<Integer, String> checkCompact(CompactType compactType) {
     Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
-        .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
+            .filterCompletedInstants().lastInstant();
+    HoodieTimeline deltaCommits = table.getActiveTimeline().getDeltaCommitTimeline();
+
+    String lastCompactionTs;
+    int deltaCommitsSinceLastCompaction = 0;
     if (lastCompaction.isPresent()) {
       lastCompactionTs = lastCompaction.get().getTimestamp();
+    } else {
+      lastCompactionTs = deltaCommits.firstInstant().get().getTimestamp();
     }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
+    if (compactType != CompactType.TIME_ELAPSED) {
+      if (lastCompaction.isPresent()) {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      } else {
+        deltaCommitsSinceLastCompaction = deltaCommits.findInstantsAfterOrEquals(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+      }
     }
+    return new Tuple2(deltaCommitsSinceLastCompaction, lastCompactionTs);
+  }
 
-    LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
-    HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
-    try {
-      SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
-      Set<HoodieFileGroupId> fgInPendingCompactionAndClustering = fileSystemView.getPendingCompactionOperations()
-          .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
-          .collect(Collectors.toSet());
-      // exclude files in pending clustering from compaction.
-      fgInPendingCompactionAndClustering.addAll(fileSystemView.getFileGroupsInPendingClustering().map(Pair::getLeft).collect(Collectors.toSet()));
-      return compactor.generateCompactionPlan(context, table, config, instantTime, fgInPendingCompactionAndClustering);
+  public boolean needCompact(CompactType compactType) {
+    boolean compactable;
+    // return deltaCommitsSinceLastCompaction and lastCompactionTs
+    Tuple2<Integer, String> threshold = checkCompact(compactType);
+    switch (compactType) {
+      case COMMIT_NUM:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1;
+        break;
+      case TIME_ELAPSED:
+        compactable = parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_OR_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            || parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      case NUM_AND_TIME:
+        compactable = config.getInlineCompactDeltaCommitMax() <= threshold._1
+            && parseToTimestamp(threshold._2) + config.getInlineCompactDeltaElapsedTimeMax() <= parseToTimestamp(instantTime);
+        break;
+      default:
+        throw new HoodieCompactionException("Unsupported compact type: " + config.getInlineCompactType());
+    }
 
-    } catch (IOException e) {
-      throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+    if (compactable) {
+      LOG.info(String.format("Scheduling compaction: %s. Delta commits found: %s times, and last compaction time is %s.",

Review comment:
       ok, added !




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (679310a) into [master](https://codecov.io/gh/apache/hudi/commit/430d4b428e7c5b325c7414a187f9cda158c2758a?el=desc) (430d4b4) will **decrease** coverage by `0.00%`.
   > The diff coverage is `0.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   - Coverage     53.54%   53.54%   -0.01%     
     Complexity     2770     2770              
   ============================================
     Files           348      348              
     Lines         16109    16110       +1     
     Branches       1643     1643              
   ============================================
     Hits           8626     8626              
   - Misses         6785     6786       +1     
     Partials        698      698              
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.37% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `55.33% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `32.94% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudispark | `65.54% <0.00%> (-0.05%)` | `0.00 <0.00> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `70.09% <ø> (ø)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...in/scala/org/apache/hudi/HoodieStreamingSink.scala](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2h1ZGkvSG9vZGllU3RyZWFtaW5nU2luay5zY2FsYQ==) | `23.76% <0.00%> (-0.24%)` | `10.00 <0.00> (ø)` | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (c2a695a) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **decrease** coverage by `40.58%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #2260       +/-   ##
   ============================================
   - Coverage     50.27%   9.68%   -40.59%     
   + Complexity     2990      48     -2942     
   ============================================
     Files           410      53      -357     
     Lines         18406    1930    -16476     
     Branches       1885     230     -1655     
   ============================================
   - Hits           9253     187     -9066     
   + Misses         8395    1730     -6665     
   + Partials        758      13      -745     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `9.68% <ø> (-59.97%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...va/org/apache/hudi/utilities/IdentitySplitter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL0lkZW50aXR5U3BsaXR0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-2.00%)` | |
   | [...va/org/apache/hudi/utilities/schema/SchemaSet.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFTZXQuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-3.00%)` | |
   | [...a/org/apache/hudi/utilities/sources/RowSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUm93U291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [.../org/apache/hudi/utilities/sources/AvroSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQXZyb1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [.../org/apache/hudi/utilities/sources/JsonSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvblNvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-1.00%)` | |
   | [...rg/apache/hudi/utilities/sources/CsvDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvQ3N2REZTU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-10.00%)` | |
   | [...g/apache/hudi/utilities/sources/JsonDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkRGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | [...apache/hudi/utilities/sources/JsonKafkaSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvSnNvbkthZmthU291cmNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-6.00%)` | |
   | [...pache/hudi/utilities/sources/ParquetDFSSource.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NvdXJjZXMvUGFycXVldERGU1NvdXJjZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-5.00%)` | |
   | [...lities/schema/SchemaProviderWithPostProcessor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS11dGlsaXRpZXMvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvdXRpbGl0aWVzL3NjaGVtYS9TY2hlbWFQcm92aWRlcldpdGhQb3N0UHJvY2Vzc29yLmphdmE=) | `0.00% <0.00%> (-100.00%)` | `0.00% <0.00%> (-4.00%)` | |
   | ... and [387 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] wangxianghu commented on a change in pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
wangxianghu commented on a change in pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#discussion_r549538945



##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
##########
@@ -90,4 +99,13 @@ protected HoodieCompactionPlan scheduleCompaction() {
     }
   }
 
+  public Long parseTs(String time) {
+    Long timestamp = null;
+    try {
+      timestamp = HoodieActiveTimeline.COMMIT_FORMATTER.parse(time).getTime() / 1000;
+    } catch (ParseException e) {
+      e.printStackTrace();

Review comment:
       we'd better throw an 'HoodieCompactionException' here




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (b34c416) into [master](https://codecov.io/gh/apache/hudi/commit/b593f1062931a4d017ae8bd7dd42e47a8873a39f?el=desc) (b593f10) will **increase** coverage by `19.24%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #2260       +/-   ##
   =============================================
   + Coverage     50.27%   69.51%   +19.24%     
   + Complexity     2990      357     -2633     
   =============================================
     Files           410       53      -357     
     Lines         18406     1929    -16477     
     Branches       1885      229     -1656     
   =============================================
   - Hits           9253     1341     -7912     
   + Misses         8395      456     -7939     
   + Partials        758      132      -626     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `?` | `?` | |
   | hudiclient | `?` | `?` | |
   | hudicommon | `?` | `?` | |
   | hudiflink | `?` | `?` | |
   | hudihadoopmr | `?` | `?` | |
   | hudisparkdatasource | `?` | `?` | |
   | hudisync | `?` | `?` | |
   | huditimelineservice | `?` | `?` | |
   | hudiutilities | `69.51% <ø> (-0.14%)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...common/table/view/FileSystemViewStorageConfig.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3ZpZXcvRmlsZVN5c3RlbVZpZXdTdG9yYWdlQ29uZmlnLmphdmE=) | | | |
   | [...di-cli/src/main/java/org/apache/hudi/cli/Main.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jbGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY2xpL01haW4uamF2YQ==) | | | |
   | [...java/org/apache/hudi/common/util/ParquetUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3V0aWwvUGFycXVldFV0aWxzLmphdmE=) | | | |
   | [...rg/apache/hudi/common/fs/NoOpConsistencyGuard.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL2ZzL05vT3BDb25zaXN0ZW5jeUd1YXJkLmphdmE=) | | | |
   | [...e/hudi/common/table/log/HoodieLogFormatWriter.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL2xvZy9Ib29kaWVMb2dGb3JtYXRXcml0ZXIuamF2YQ==) | | | |
   | [...spark3/internal/HoodieDataSourceInternalTable.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3BhcmszL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL3NwYXJrMy9pbnRlcm5hbC9Ib29kaWVEYXRhU291cmNlSW50ZXJuYWxUYWJsZS5qYXZh) | | | |
   | [...sioning/clean/CleanMetadataV1MigrationHandler.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1jb21tb24vc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvY29tbW9uL3RhYmxlL3RpbWVsaW5lL3ZlcnNpb25pbmcvY2xlYW4vQ2xlYW5NZXRhZGF0YVYxTWlncmF0aW9uSGFuZGxlci5qYXZh) | | | |
   | [.../org/apache/hudi/hive/NonPartitionedExtractor.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zeW5jL2h1ZGktaGl2ZS1zeW5jL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9odWRpL2hpdmUvTm9uUGFydGl0aW9uZWRFeHRyYWN0b3IuamF2YQ==) | | | |
   | [.../main/scala/org/apache/hudi/HoodieSparkUtils.scala](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1zcGFyay1kYXRhc291cmNlL2h1ZGktc3Bhcmsvc3JjL21haW4vc2NhbGEvb3JnL2FwYWNoZS9odWRpL0hvb2RpZVNwYXJrVXRpbHMuc2NhbGE=) | | | |
   | [...g/apache/hudi/hadoop/HoodieParquetInputFormat.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL0hvb2RpZVBhcnF1ZXRJbnB1dEZvcm1hdC5qYXZh) | | | |
   | ... and [351 more](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree-more) | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] codecov-io edited a comment on pull request #2260: [HUDI-1381] Schedule compaction based on time elapsed

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on pull request #2260:
URL: https://github.com/apache/hudi/pull/2260#issuecomment-729530724


   # [Codecov](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=h1) Report
   > Merging [#2260](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=desc) (6ff932e) into [master](https://codecov.io/gh/apache/hudi/commit/e807bb895e5a31a40f51b2c2bd9b732d26bebd46?el=desc) (e807bb8) will **increase** coverage by `0.02%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/hudi/pull/2260/graphs/tree.svg?width=650&height=150&src=pr&token=VTTXabwbs2)](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #2260      +/-   ##
   ============================================
   + Coverage     52.20%   52.23%   +0.02%     
   - Complexity     2659     2662       +3     
   ============================================
     Files           335      335              
     Lines         14981    14981              
     Branches       1505     1506       +1     
   ============================================
   + Hits           7821     7825       +4     
   + Misses         6535     6533       -2     
   + Partials        625      623       -2     
   ```
   
   | Flag | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | hudicli | `38.83% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiclient | `100.00% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudicommon | `54.76% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudihadoopmr | `33.52% <ø> (+0.23%)` | `0.00 <ø> (ø)` | |
   | huditimelineservice | `65.30% <ø> (ø)` | `0.00 <ø> (ø)` | |
   | hudiutilities | `69.65% <ø> (ø)` | `0.00 <ø> (ø)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/hudi/pull/2260?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [...ache/hudi/hadoop/utils/HoodieInputFormatUtils.java](https://codecov.io/gh/apache/hudi/pull/2260/diff?src=pr&el=tree#diff-aHVkaS1oYWRvb3AtbXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2h1ZGkvaGFkb29wL3V0aWxzL0hvb2RpZUlucHV0Rm9ybWF0VXRpbHMuamF2YQ==) | `50.32% <0.00%> (+2.61%)` | `23.00% <0.00%> (+3.00%)` | |
   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org