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 2022/07/02 06:14:43 UTC

[GitHub] [hudi] XuQianJin-Stars opened a new pull request, #6025: [HUDI-4351] Improve HoodieFlinkCompactor

XuQianJin-Stars opened a new pull request, #6025:
URL: https://github.com/apache/hudi/pull/6025

   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contribute/how-to-contribute before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   *(For example: This pull request adds quick-start document.)*
   
   ## Brief change log
   
   *(for example:)*
     - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [ ] 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.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6025:
URL: https://github.com/apache/hudi/pull/6025#discussion_r912431748


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.java:
##########
@@ -48,47 +60,154 @@
  *   as the instant time.</li>
  * </ul>
  */
-public class CompactionPlanSourceFunction extends AbstractRichFunction implements SourceFunction<CompactionPlanEvent> {
+public class CompactionPlanSourceFunction
+    extends AbstractRichFunction implements SourceFunction<CompactionPlanEvent> {
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionPlanSourceFunction.class);
+
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * The interval between consecutive path scans.
+   */
+  private final long interval;
 
-  protected static final Logger LOG = LoggerFactory.getLogger(CompactionPlanSourceFunction.class);
+  private volatile boolean isRunning = true;
+
+  private final Configuration conf;
+
+  protected HoodieFlinkWriteClient writeClient;
 
   /**
-   * Compaction instant time.
+   * The hoodie table.
    */
-  private final String compactionInstantTime;
+  private transient HoodieFlinkTable<?> table;
 
   /**
-   * The compaction plan.
+   * The path to monitor.
    */
-  private final HoodieCompactionPlan compactionPlan;
+  private final transient Path path;
+
+  private final Boolean isStreamingMode;
 
-  public CompactionPlanSourceFunction(HoodieCompactionPlan compactionPlan, String compactionInstantTime) {
-    this.compactionPlan = compactionPlan;
-    this.compactionInstantTime = compactionInstantTime;
+  public CompactionPlanSourceFunction(
+      Configuration conf,
+      String path,
+      Boolean isStreamingMode) {
+    this.conf = conf;
+    this.path = new Path(path);
+    this.isStreamingMode = isStreamingMode;
+    this.interval = conf.getInteger(FlinkOptions.COMPACTION_STREAMING_CHECK_INTERVAL);
   }
 
   @Override
   public void open(Configuration parameters) throws Exception {
-    // no operation
+    super.open(parameters);
+    if (writeClient == null) {
+      this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
+    }
+    this.table = this.writeClient.getHoodieTable();
   }
 
   @Override
-  public void run(SourceContext sourceContext) throws Exception {
-    List<CompactionOperation> operations = this.compactionPlan.getOperations().stream()
+  public void run(SourceContext<CompactionPlanEvent> context) throws Exception {
+    if (isStreamingMode) {
+      while (isRunning) {
+        monitorCompactionPlan(context);
+        TimeUnit.SECONDS.sleep(interval);
+      }
+    } else {
+      monitorCompactionPlan(context);
+    }
+  }
+
+  public void monitorCompactionPlan(SourceContext<CompactionPlanEvent> context) throws IOException {
+    table.getMetaClient().reloadActiveTimeline();
+
+    // checks the compaction plan and do compaction.
+    if (OptionsResolver.needsScheduleCompaction(conf)) {
+      Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(table.getMetaClient());
+      if (compactionInstantTimeOption.isPresent()) {
+        boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
+        if (!scheduled) {
+          // do nothing.
+          LOG.info("No compaction plan for this job ");
+          return;
+        }
+        table.getMetaClient().reloadActiveTimeline();
+      }
+    }
+
+    // fetch the instant based on the configured execution sequence
+    String compactionSeq = conf.getString(FlinkOptions.COMPACTION_SEQUENCE);
+    HoodieTimeline timeline = table.getActiveTimeline().filterPendingCompactionTimeline();
+    Option<HoodieInstant> requested = CompactionUtil.isLIFO(compactionSeq) ? timeline.lastInstant() : timeline.firstInstant();
+    if (!requested.isPresent()) {
+      // do nothing.
+      LOG.info("No compaction plan scheduled, turns on the compaction plan schedule with --schedule option");
+      return;
+    }
+
+    String compactionInstantTime = requested.get().getTimestamp();
+
+    HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
+    if (timeline.containsInstant(inflightInstant)) {
+      LOG.info("Rollback inflight compaction instant: [" + compactionInstantTime + "]");
+      table.rollbackInflightCompaction(inflightInstant);
+      table.getMetaClient().reloadActiveTimeline();
+    }
+
+    // generate compaction plan
+    // should support configurable commit metadata
+    HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(
+        table.getMetaClient(), compactionInstantTime);
+
+    if (compactionPlan == null || (compactionPlan.getOperations() == null)
+        || (compactionPlan.getOperations().isEmpty())) {
+      // No compaction plan, do nothing and return.
+      LOG.info("No compaction plan for instant " + compactionInstantTime);
+      return;
+    }
+
+    HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
+    HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
+    if (!pendingCompactionTimeline.containsInstant(instant)) {

Review Comment:
   Is it necessary to start a separate scheduling for compaction ? There is only possible compaction plans when an instant is committed successfully, seems more reasonable to schedule the compaction on ckp success event, based on the fact users may have custom ckp interval config.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1172999010

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3136c4d3478da9fea0e5bf45a23c226644b3ec10 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672) 
   * b0115c3e076eca9c3d2a39d8e070f7c7fb387536 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681) 
   * 845c4f50c09b97b530ee913bf5fee52c837ccc21 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1174961958

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684",
       "triggerID" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f2d5ea5872e47c04c106f614d79385d85a0c4a13 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684) 
   * c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1173010178

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684",
       "triggerID" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 845c4f50c09b97b530ee913bf5fee52c837ccc21 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682) 
   * f2d5ea5872e47c04c106f614d79385d85a0c4a13 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1176315609

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684",
       "triggerID" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9731",
       "triggerID" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a58c987ef36c1ec09df202a9f57c589e4e38e16b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9748",
       "triggerID" : "a58c987ef36c1ec09df202a9f57c589e4e38e16b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9731) 
   * a58c987ef36c1ec09df202a9f57c589e4e38e16b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9748) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6025:
URL: https://github.com/apache/hudi/pull/6025#discussion_r913753917


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -99,6 +94,12 @@ public void start(boolean serviceMode) throws Exception {
     }
   }
 
+  public void shutDown() throws Exception {
+    if (compactionScheduleService != null) {

Review Comment:
   What did you mean for async compaction scheduling ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] garyli1019 commented on a diff in pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on code in PR #6025:
URL: https://github.com/apache/hudi/pull/6025#discussion_r914610339


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -99,6 +94,12 @@ public void start(boolean serviceMode) throws Exception {
     }
   }
 
+  public void shutDown() throws Exception {
+    if (compactionScheduleService != null) {

Review Comment:
   scheduling compaction on the batch job when the streaming job is ingesting data into the table.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1173017152

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684",
       "triggerID" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f2d5ea5872e47c04c106f614d79385d85a0c4a13 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] TJX2014 commented on a diff in pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
TJX2014 commented on code in PR #6025:
URL: https://github.com/apache/hudi/pull/6025#discussion_r912660211


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.java:
##########
@@ -48,47 +60,154 @@
  *   as the instant time.</li>
  * </ul>
  */
-public class CompactionPlanSourceFunction extends AbstractRichFunction implements SourceFunction<CompactionPlanEvent> {
+public class CompactionPlanSourceFunction
+    extends AbstractRichFunction implements SourceFunction<CompactionPlanEvent> {
+  private static final Logger LOG = LoggerFactory.getLogger(CompactionPlanSourceFunction.class);
+
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * The interval between consecutive path scans.
+   */
+  private final long interval;
 
-  protected static final Logger LOG = LoggerFactory.getLogger(CompactionPlanSourceFunction.class);
+  private volatile boolean isRunning = true;
+
+  private final Configuration conf;
+
+  protected HoodieFlinkWriteClient writeClient;
 
   /**
-   * Compaction instant time.
+   * The hoodie table.
    */
-  private final String compactionInstantTime;
+  private transient HoodieFlinkTable<?> table;
 
   /**
-   * The compaction plan.
+   * The path to monitor.
    */
-  private final HoodieCompactionPlan compactionPlan;
+  private final transient Path path;
+
+  private final Boolean isStreamingMode;
 
-  public CompactionPlanSourceFunction(HoodieCompactionPlan compactionPlan, String compactionInstantTime) {
-    this.compactionPlan = compactionPlan;
-    this.compactionInstantTime = compactionInstantTime;
+  public CompactionPlanSourceFunction(
+      Configuration conf,
+      String path,
+      Boolean isStreamingMode) {
+    this.conf = conf;
+    this.path = new Path(path);
+    this.isStreamingMode = isStreamingMode;
+    this.interval = conf.getInteger(FlinkOptions.COMPACTION_STREAMING_CHECK_INTERVAL);
   }
 
   @Override
   public void open(Configuration parameters) throws Exception {
-    // no operation
+    super.open(parameters);
+    if (writeClient == null) {
+      this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
+    }
+    this.table = this.writeClient.getHoodieTable();
   }
 
   @Override
-  public void run(SourceContext sourceContext) throws Exception {
-    List<CompactionOperation> operations = this.compactionPlan.getOperations().stream()
+  public void run(SourceContext<CompactionPlanEvent> context) throws Exception {
+    if (isStreamingMode) {
+      while (isRunning) {
+        monitorCompactionPlan(context);
+        TimeUnit.SECONDS.sleep(interval);
+      }
+    } else {
+      monitorCompactionPlan(context);
+    }
+  }
+
+  public void monitorCompactionPlan(SourceContext<CompactionPlanEvent> context) throws IOException {
+    table.getMetaClient().reloadActiveTimeline();
+
+    // checks the compaction plan and do compaction.
+    if (OptionsResolver.needsScheduleCompaction(conf)) {
+      Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(table.getMetaClient());
+      if (compactionInstantTimeOption.isPresent()) {
+        boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
+        if (!scheduled) {
+          // do nothing.
+          LOG.info("No compaction plan for this job ");
+          return;
+        }
+        table.getMetaClient().reloadActiveTimeline();
+      }
+    }
+
+    // fetch the instant based on the configured execution sequence
+    String compactionSeq = conf.getString(FlinkOptions.COMPACTION_SEQUENCE);
+    HoodieTimeline timeline = table.getActiveTimeline().filterPendingCompactionTimeline();
+    Option<HoodieInstant> requested = CompactionUtil.isLIFO(compactionSeq) ? timeline.lastInstant() : timeline.firstInstant();
+    if (!requested.isPresent()) {
+      // do nothing.
+      LOG.info("No compaction plan scheduled, turns on the compaction plan schedule with --schedule option");
+      return;
+    }
+
+    String compactionInstantTime = requested.get().getTimestamp();
+
+    HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
+    if (timeline.containsInstant(inflightInstant)) {
+      LOG.info("Rollback inflight compaction instant: [" + compactionInstantTime + "]");
+      table.rollbackInflightCompaction(inflightInstant);
+      table.getMetaClient().reloadActiveTimeline();

Review Comment:
   It will rollback it self in the next round.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] garyli1019 commented on a diff in pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
garyli1019 commented on code in PR #6025:
URL: https://github.com/apache/hudi/pull/6025#discussion_r913431715


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java:
##########
@@ -507,6 +507,20 @@ private FlinkOptions() {
       .defaultValue(4) // default WRITE_TASKS * COMPACTION_DELTA_COMMITS * 0.2 (assumes 5 commits generate one bucket)
       .withDescription("Parallelism of tasks that do actual compaction, default is 4");
 
+  public static final ConfigOption<String> COMPACTION_SEQUENCE = ConfigOptions
+      .key("compaction.sequence")
+      .stringType()
+      .defaultValue("LIFO") // default WRITE_TASKS * COMPACTION_DELTA_COMMITS * 0.2 (assumes 5 commits generate one bucket)

Review Comment:
   LIFO is not recommended. It's possible the oldest plan never get executed and the active timeline get longer and longer. 



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -99,6 +94,12 @@ public void start(boolean serviceMode) throws Exception {
     }
   }
 
+  public void shutDown() throws Exception {
+    if (compactionScheduleService != null) {

Review Comment:
   didn't check this code for a while. Did we support async compaction scheduling already?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1176310337

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684",
       "triggerID" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9731",
       "triggerID" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a58c987ef36c1ec09df202a9f57c589e4e38e16b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a58c987ef36c1ec09df202a9f57c589e4e38e16b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9731) 
   * a58c987ef36c1ec09df202a9f57c589e4e38e16b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1173004729

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3136c4d3478da9fea0e5bf45a23c226644b3ec10 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672) 
   * b0115c3e076eca9c3d2a39d8e070f7c7fb387536 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681) 
   * 845c4f50c09b97b530ee913bf5fee52c837ccc21 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682) 
   * f2d5ea5872e47c04c106f614d79385d85a0c4a13 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1174965566

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684",
       "triggerID" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9731",
       "triggerID" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f2d5ea5872e47c04c106f614d79385d85a0c4a13 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684) 
   * c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9731) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1175219792

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684",
       "triggerID" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9731",
       "triggerID" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9731) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] XuQianJin-Stars closed pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars closed pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor
URL: https://github.com/apache/hudi/pull/6025


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1172844426

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9b3fc9d9e9767eee22c18ad4010788270828c42a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1172999390

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3136c4d3478da9fea0e5bf45a23c226644b3ec10 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672) 
   * b0115c3e076eca9c3d2a39d8e070f7c7fb387536 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681) 
   * 845c4f50c09b97b530ee913bf5fee52c837ccc21 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1172989655

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3136c4d3478da9fea0e5bf45a23c226644b3ec10 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672) 
   * b0115c3e076eca9c3d2a39d8e070f7c7fb387536 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1173005046

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684",
       "triggerID" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3136c4d3478da9fea0e5bf45a23c226644b3ec10 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672) 
   * b0115c3e076eca9c3d2a39d8e070f7c7fb387536 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681) 
   * 845c4f50c09b97b530ee913bf5fee52c837ccc21 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682) 
   * f2d5ea5872e47c04c106f614d79385d85a0c4a13 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1172874705

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3136c4d3478da9fea0e5bf45a23c226644b3ec10 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1172849762

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9b3fc9d9e9767eee22c18ad4010788270828c42a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] danny0405 commented on a diff in pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #6025:
URL: https://github.com/apache/hudi/pull/6025#discussion_r915527821


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -99,6 +94,12 @@ public void start(boolean serviceMode) throws Exception {
     }
   }
 
+  public void shutDown() throws Exception {
+    if (compactionScheduleService != null) {

Review Comment:
   No, we do not support that ~



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1172858565

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9b3fc9d9e9767eee22c18ad4010788270828c42a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670) 
   * 3136c4d3478da9fea0e5bf45a23c226644b3ec10 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1172989284

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3136c4d3478da9fea0e5bf45a23c226644b3ec10 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672) 
   * b0115c3e076eca9c3d2a39d8e070f7c7fb387536 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1172852693

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9b3fc9d9e9767eee22c18ad4010788270828c42a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670) 
   * 3136c4d3478da9fea0e5bf45a23c226644b3ec10 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1172850430

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9b3fc9d9e9767eee22c18ad4010788270828c42a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670) 
   * 3136c4d3478da9fea0e5bf45a23c226644b3ec10 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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


[GitHub] [hudi] hudi-bot commented on pull request #6025: [HUDI-4351] Improve HoodieFlinkCompactor

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #6025:
URL: https://github.com/apache/hudi/pull/6025#issuecomment-1176448918

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9670",
       "triggerID" : "9b3fc9d9e9767eee22c18ad4010788270828c42a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9672",
       "triggerID" : "3136c4d3478da9fea0e5bf45a23c226644b3ec10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9681",
       "triggerID" : "b0115c3e076eca9c3d2a39d8e070f7c7fb387536",
       "triggerType" : "PUSH"
     }, {
       "hash" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9682",
       "triggerID" : "845c4f50c09b97b530ee913bf5fee52c837ccc21",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9684",
       "triggerID" : "f2d5ea5872e47c04c106f614d79385d85a0c4a13",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9731",
       "triggerID" : "c71c6e8ea76e26fc528daa2ed5f05a6fc99dfe7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a58c987ef36c1ec09df202a9f57c589e4e38e16b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9748",
       "triggerID" : "a58c987ef36c1ec09df202a9f57c589e4e38e16b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a58c987ef36c1ec09df202a9f57c589e4e38e16b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9748) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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