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/09/01 09:58:38 UTC

[GitHub] [hudi] voonhous opened a new pull request, #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

voonhous opened a new pull request, #6566:
URL: https://github.com/apache/hudi/pull/6566

   1. Allow rollbacks if required during clustering
   2. Allow size to be defined in Long instead of Integer
   3. Fix bug where clustering will produce files of 120MB in the same filegroup
   4. Added clean task
   5. Fix scheduling config to be consistent with that with compaction
   6. Fix filter mode getting ignored issue
   7. Add --instant-time parameter
   8. Prevent no execute() calls exception from being thrown (clustering & compaction)
   
   ### Change Logs
   
   _Describe context and summary for this change. Highlight if any code was copied._
   
   ### Impact
   
   _Describe any public API or user-facing feature change or any performance impact._
   
   **Risk level: none | low | medium | high**
   
   _Choose one. If medium or high, explain what verification was done to mitigate the risks._
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -215,82 +226,126 @@ protected Pair<CompletableFuture, ExecutorService> startService() {
       }, executor), executor);
     }
 
+    /**
+     * Follows the same execution methodology of HoodieFlinkCompactor, where only one clustering job
+     * is allowed to be executed at any point in time.
+     * <p>
+     * If there is an inflight clustering job, it will be rolled back and re-attempted.
+     * <p>
+     * A clustering plan will be generated if `schedule` is true.
+     *
+     * @throws Exception
+     * @see HoodieFlinkCompactor
+     */
     private void cluster() throws Exception {
       table.getMetaClient().reloadActiveTimeline();
 
-      // judges whether there are operations
-      // to compute the clustering instant time and exec clustering.
       if (cfg.schedule) {
+        // create a clustering plan on the timeline
         ClusteringUtil.validateClusteringScheduling(conf);
-        String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();
-        boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());
+
+        String clusteringInstantTime = cfg.clusteringInstantTime != null ? cfg.clusteringInstantTime
+            : HoodieActiveTimeline.createNewInstantTime();
+
+        LOG.info("Creating a clustering plan for instant [" + clusteringInstantTime + "]");
+        boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime,
+            Option.empty());
         if (!scheduled) {
           // do nothing.
           LOG.info("No clustering plan for this job");
+          executeDummyPipeline();
           return;
         }
         table.getMetaClient().reloadActiveTimeline();
       }
 
       // fetch the instant based on the configured execution sequence
-      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()).stream()
-          .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).collect(Collectors.toList());
+      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(
+          table.getMetaClient());
       if (instants.isEmpty()) {
         // do nothing.
-        LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option");
+        LOG.info(
+            "No clustering plan scheduled, turns on the clustering plan schedule with --schedule option");
+        executeDummyPipeline();
         return;
       }
 
-      HoodieInstant clusteringInstant = CompactionUtil.isLIFO(cfg.clusteringSeq) ? instants.get(instants.size() - 1) : instants.get(0);
+      HoodieInstant reqClusteringInstant;
+      if (cfg.clusteringInstantTime != null) {
+        List<HoodieInstant> reqHoodieInstant = instants
+            .stream()
+            .filter(i -> i.getTimestamp().equals(cfg.clusteringInstantTime))

Review Comment:
   use `anyMatch` for checking



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   What do you mean for `no execute() calls` then ?



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

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

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


[GitHub] [hudi] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   @danny0405 Error message will not happen in service mode. 
   
   When running under service mode, the job will wait for a plan to be created.



-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -116,14 +131,57 @@ public class FlinkClusteringConfig extends Configuration {
       description = "Min clustering interval of async clustering service, default 10 minutes")
   public Integer minClusteringIntervalSeconds = 600;
 
+  @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
+          + "(using the CLI parameter \"--props\") can also be passed command line using this parameter.")
+  public List<String> configs = new ArrayList<>();
+
+  @Parameter(names = {"--props"}, description = "Path to properties file on localfs or dfs, with configurations for "
+          + "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are "
+          + "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer"
+          + "to individual classes, for supported properties.")
+  public String propsFilePath = "";
+
+  /**
+   * Read config from properties file (`--props` option) and cmd line (`--hoodie-conf` option).
+   */
+  public static DFSPropertiesConfiguration readConfig(org.apache.hadoop.conf.Configuration hadoopConfig, Path cfgPath, List<String> overriddenProps) {
+    DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(hadoopConfig, cfgPath);
+    try {
+      if (!overriddenProps.isEmpty()) {
+        conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps))));
+      }
+    } catch (IOException ioe) {
+      throw new HoodieIOException("Unexpected error adding config overrides", ioe);
+    }
+
+    return conf;
+  }
+
+  public static TypedProperties buildProperties(List<String> props) {
+    TypedProperties properties = DFSPropertiesConfiguration.getGlobalProps();

Review Comment:
   Understood. I rewrote this method as the one in StreamerUtil is buggy and will ignore parameters passed in by `--hoodie-conf` if `--props` is empty.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -215,37 +217,63 @@ protected Pair<CompletableFuture, ExecutorService> startService() {
       }, executor), executor);
     }
 
+    /**
+     * Follows the same execution methodology of HoodieFlinkCompactor, where only one clustering job is allowed to be
+     * executed at any point in time.
+     * <p>
+     * If there is an inflight clustering job, it will be rolled back and re-attempted.
+     * <p>
+     * A clustering plan will be generated if `schedule` is true.
+     *
+     * @throws Exception
+     * @see HoodieFlinkCompactor
+     */
     private void cluster() throws Exception {
       table.getMetaClient().reloadActiveTimeline();
 
-      // judges whether there are operations
-      // to compute the clustering instant time and exec clustering.
       if (cfg.schedule) {
+        // create a clustering plan on the timeline
         ClusteringUtil.validateClusteringScheduling(conf);
-        String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();
+
+        String clusteringInstantTime = cfg.clusteringInstantTime != null ? cfg.clusteringInstantTime
+            : HoodieActiveTimeline.createNewInstantTime();
+
+        LOG.info("Creating a clustering plan for instant [" + clusteringInstantTime + "]");
         boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());
         if (!scheduled) {
           // do nothing.
           LOG.info("No clustering plan for this job");
+          executeDummyPipeline();
           return;
         }
         table.getMetaClient().reloadActiveTimeline();
       }
 
       // fetch the instant based on the configured execution sequence
-      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()).stream()
-          .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).collect(Collectors.toList());
+      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient());
       if (instants.isEmpty()) {
         // do nothing.
         LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option");
+        executeDummyPipeline();
         return;
       }
 
-      HoodieInstant clusteringInstant = CompactionUtil.isLIFO(cfg.clusteringSeq) ? instants.get(instants.size() - 1) : instants.get(0);
+      HoodieInstant reqClusteringInstant;
+      if (cfg.clusteringInstantTime != null) {

Review Comment:
   Let's name it back to `clusteringInstant`



-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   While I have yet to test it on service mode, from my understanding of the code:
   
   Given that an error is already thrown here (non service mode):
   ```java
   try {
           compactionScheduleService.compact();
         } catch (Exception e) {
           LOG.error("Got error running delta sync once. Shutting down", e);
           throw e;
         } finally {
           LOG.info("Shut down hoodie flink compactor");
         }
   ```
   
   The error will be thrown as a `HoodieException` in the same service mode execution block.
   
   ```java
   try {
     compact();
     Thread.sleep(cfg.minCompactionIntervalSeconds * 1000);
   } catch (Exception e) {
     LOG.error("Shutting down compaction service due to exception", e);
     error = true;
     throw new HoodieException(e.getMessage(), e);
   }
   ```
   
   The error logs in text form in non-service mode execution:
   ```java
   2022-09-06 11:59:45,633 INFO  org.apache.hudi.common.table.HoodieTableMetaClient           [] - Loading HoodieTableMetaClient from hdfs://hudi_mor_compaction_test/
   2022-09-06 11:59:45,638 INFO  org.apache.hudi.common.table.HoodieTableConfig               [] - Loading table properties from hdfs://hudi_mor_compaction_test/.hoodie/hoodie.properties
   2022-09-06 11:59:45,643 INFO  org.apache.hudi.common.table.HoodieTableMetaClient           [] - Finished Loading Table of type MERGE_ON_READ(version=1, baseFileFormat=PARQUET) from hdfs://hudi_mor_compaction_test/
   2022-09-06 11:59:45,643 INFO  org.apache.hudi.common.table.HoodieTableMetaClient           [] - Loading Active commit timeline for hdfs://hudi_mor_compaction_test/
   2022-09-06 11:59:46,012 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20220906115739354__deltacommit__INFLIGHT]}
   2022-09-06 11:59:46,018 INFO  org.apache.hudi.common.table.view.FileSystemViewManager      [] - Creating View Manager with storage type :REMOTE_FIRST
   2022-09-06 11:59:46,018 INFO  org.apache.hudi.common.table.view.FileSystemViewManager      [] - Creating remote first table view
   2022-09-06 11:59:46,020 INFO  org.apache.hudi.sink.compact.HoodieFlinkCompactor            [] - Hoodie Flink Compactor running only single round
   2022-09-06 11:59:46,361 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20220906115739354__deltacommit__INFLIGHT]}
   2022-09-06 11:59:46,363 INFO  org.apache.hudi.client.RunsTableService                      [] - No compaction plan scheduled, turns on the compaction plan schedule with --schedule option
   2022-09-06 11:59:46,363 INFO  org.apache.hudi.sink.compact.HoodieFlinkCompactor            [] - Shut down hoodie flink compactor
   2022-09-06 11:59:46,420 ERROR org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap [] - Application failed unexpectedly. Killing cluster...
   java.util.concurrent.CompletionException: org.apache.flink.client.deployment.application.ApplicationExecutionException: The application contains no execute() calls.
   	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292) ~[?:1.8.0_232]
   	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308) ~[?:1.8.0_232]
   	at java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:957) ~[?:1.8.0_232]
   	at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:940) ~[?:1.8.0_232]
   	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488) ~[?:1.8.0_232]
   	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1990) ~[?:1.8.0_232]
   	at org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap.runApplicationEntryPoint(ApplicationDispatcherBootstrap.java:263) ~[flink-dist_2.11-1.13.17.jar:1.13.17]
   	at org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap.lambda$runApplicationAsync$1(ApplicationDispatcherBootstrap.java:219) ~[flink-dist_2.11-1.13.17.jar:1.13.17]
   	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_232]
   	at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_232]
   	at org.apache.flink.runtime.concurrent.akka.ActorSystemScheduledExecutorAdapter$ScheduledFutureTask.run(ActorSystemScheduledExecutorAdapter.java:159) [flink-dist_2.11-1.13.17.jar:1.13.17]
   	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40) [flink-dist_2.11-1.13.17.jar:1.13.17]
   	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:44) [flink-dist_2.11-1.13.17.jar:1.13.17]
   	at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) [flink-dist_2.11-1.13.17.jar:1.13.17]
   	at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) [flink-dist_2.11-1.13.17.jar:1.13.17]
   	at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) [flink-dist_2.11-1.13.17.jar:1.13.17]
   	at akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) [flink-dist_2.11-1.13.17.jar:1.13.17]
   Caused by: org.apache.flink.client.deployment.application.ApplicationExecutionException: The application contains no execute() calls.
   	at org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap.runApplicationEntryPoint(ApplicationDispatcherBootstrap.java:257) ~[flink-dist_2.11-1.13.17.jar:1.13.17]
   	... 10 more
   2022-09-06 11:59:46,427 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint        [] - Shutting YarnApplicationClusterEntryPoint down with application status FAILED. Diagnostics null.
   2022-09-06 11:59:46,429 INFO  org.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint [] - Shutting down rest endpoint.
   ```



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * 1709f71ae9494da4d7ca6b9c62ac97cd11dd8046 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146) 
   * f54c41a18ee1070249aafd9c81830699b360d0b9 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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a53950c478652bf65a959fa1f6f3930a84635d22 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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * 4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088) 
   
   <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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -69,13 +83,14 @@ public class FlinkClusteringConfig extends Configuration {
       required = false)
   public Integer archiveMaxCommits = 30;
 
-  @Parameter(names = {"--schedule", "-sc"}, description = "Not recommended. Schedule the clustering plan in this job.\n"
-      + "There is a risk of losing data when scheduling clustering outside the writer job.\n"
-      + "Scheduling clustering in the writer job and only let this job do the clustering execution is recommended.\n"
-      + "Default is true", required = false)
-  public Boolean schedule = true;
+  @Parameter(names = {"--schedule", "-sc"}, description = "Schedule the clustering plan in this job.\n"
+      + "Default is false", required = false)
+  public Boolean schedule = false;
+
+  @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time")
+  public String clusteringInstantTime = null;

Review Comment:
   Can we explain the use case why explicit instant time is needed for clustering execution ?



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -163,7 +167,8 @@ public static class AsyncClusteringService extends HoodieAsyncTableService {
      */
     private final ExecutorService executor;
 
-    public AsyncClusteringService(FlinkClusteringConfig cfg, Configuration conf, StreamExecutionEnvironment env) throws Exception {
+    public AsyncClusteringService(FlinkClusteringConfig cfg, Configuration conf,
+        StreamExecutionEnvironment env) throws Exception {

Review Comment:
   Unnecessary change.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   Seems it is related with the way to submit the job, does the same error happens for CLI submission(for example, the service mode), if it is not, i would suggest to remove the dummy pipeline to avoid confusion.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -69,13 +83,14 @@ public class FlinkClusteringConfig extends Configuration {
       required = false)
   public Integer archiveMaxCommits = 30;
 
-  @Parameter(names = {"--schedule", "-sc"}, description = "Not recommended. Schedule the clustering plan in this job.\n"
-      + "There is a risk of losing data when scheduling clustering outside the writer job.\n"
-      + "Scheduling clustering in the writer job and only let this job do the clustering execution is recommended.\n"
-      + "Default is true", required = false)
-  public Boolean schedule = true;
+  @Parameter(names = {"--schedule", "-sc"}, description = "Schedule the clustering plan in this job.\n"
+      + "Default is false", required = false)
+  public Boolean schedule = false;
+
+  @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time")
+  public String clusteringInstantTime = null;

Review Comment:
   Let's rename `--instant-time` to `--instant`



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -116,14 +131,57 @@ public class FlinkClusteringConfig extends Configuration {
       description = "Min clustering interval of async clustering service, default 10 minutes")
   public Integer minClusteringIntervalSeconds = 600;
 
+  @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
+          + "(using the CLI parameter \"--props\") can also be passed command line using this parameter.")
+  public List<String> configs = new ArrayList<>();
+
+  @Parameter(names = {"--props"}, description = "Path to properties file on localfs or dfs, with configurations for "
+          + "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are "
+          + "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer"
+          + "to individual classes, for supported properties.")
+  public String propsFilePath = "";
+
+  /**
+   * Read config from properties file (`--props` option) and cmd line (`--hoodie-conf` option).
+   */
+  public static DFSPropertiesConfiguration readConfig(org.apache.hadoop.conf.Configuration hadoopConfig, Path cfgPath, List<String> overriddenProps) {
+    DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(hadoopConfig, cfgPath);
+    try {
+      if (!overriddenProps.isEmpty()) {
+        conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps))));
+      }
+    } catch (IOException ioe) {
+      throw new HoodieIOException("Unexpected error adding config overrides", ioe);
+    }
+
+    return conf;
+  }
+
+  public static TypedProperties buildProperties(List<String> props) {
+    TypedProperties properties = DFSPropertiesConfiguration.getGlobalProps();

Review Comment:
   We can also move this clazz into `StreamerUtil`, and please fire another JIRA to refactor the method `readConfig` and `buildProperties` to move them into clazz DFSPropertiesConfiguration.



-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -211,28 +214,36 @@ private void compact() throws Exception {
 
       // checks the compaction plan and do compaction.
       if (cfg.schedule) {
-        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient);
+        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(
+            metaClient);
         if (compactionInstantTimeOption.isPresent()) {
-          boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
+          boolean scheduled = writeClient.scheduleCompactionAtInstant(
+              compactionInstantTimeOption.get(), Option.empty());

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.

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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -279,18 +310,18 @@ private void cluster() throws Exception {
         // exceptionally.
 
         // clean the clustering plan in auxiliary path and cancels the clustering.
-
         LOG.warn("The clustering plan was fetched through the auxiliary path(.tmp) but not the meta path(.hoodie).\n"
             + "Clean the clustering plan in auxiliary path and cancels the clustering");
         CompactionUtil.cleanInstant(table.getMetaClient(), instant);
+        executeDummyPipeline();
         return;
       }
 
       // get clusteringParallelism.
       int clusteringParallelism = conf.getInteger(FlinkOptions.CLUSTERING_TASKS) == -1
           ? clusteringPlan.getInputGroups().size() : conf.getInteger(FlinkOptions.CLUSTERING_TASKS);
 
-      // Mark instant as clustering inflight
+      // mark instant as clustering inflight

Review Comment:
   Revert all the unnecessary change.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   Should we just stop submit the job to cluster then ?



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

To unsubscribe, e-mail: 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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * 4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088) 
   * a2dcd81f74603e88c4db895900d43eee6702a6da 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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * 4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088) 
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * a9c0e105f4f90f8403649a3a01bddaf3db23ed03 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105) 
   
   <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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * e8100c4d856971de8dd42ba239a4f029d6ce676e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115) 
   * 257a2f2acf08448c082c89510cd731b4d8f1b877 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130) 
   
   <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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154",
       "triggerID" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1768dc61360813b1240679c5f89d037c70135855",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1768dc61360813b1240679c5f89d037c70135855",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * f54c41a18ee1070249aafd9c81830699b360d0b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154) 
   * 38b7cfffeb455c6745b7d23445e48292567e7b9a UNKNOWN
   * d474c352d06287240f1b26c4bb31931eb216717c UNKNOWN
   * 1768dc61360813b1240679c5f89d037c70135855 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] danny0405 commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -69,13 +83,14 @@ public class FlinkClusteringConfig extends Configuration {
       required = false)
   public Integer archiveMaxCommits = 30;
 
-  @Parameter(names = {"--schedule", "-sc"}, description = "Not recommended. Schedule the clustering plan in this job.\n"
-      + "There is a risk of losing data when scheduling clustering outside the writer job.\n"
-      + "Scheduling clustering in the writer job and only let this job do the clustering execution is recommended.\n"
-      + "Default is true", required = false)
-  public Boolean schedule = true;
+  @Parameter(names = {"--schedule", "-sc"}, description = "Schedule the clustering plan in this job.\n"
+      + "Default is false", required = false)
+  public Boolean schedule = false;
+
+  @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time")
+  public String clusteringInstantTime = null;

Review Comment:
   Fine, we can keep that if spark already name it like this.



-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -259,15 +287,18 @@ private void cluster() throws Exception {
       if (!clusteringPlanOption.isPresent()) {
         // do nothing.
         LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option");
+        executeDummyPipeline();
         return;
       }
 
+      clusteringInstant = clusteringPlanOption.get().getLeft();
       HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight();

Review Comment:
   No special reasons, I will remove it.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a53950c478652bf65a959fa1f6f3930a84635d22 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086) 
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * 4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b 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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154",
       "triggerID" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * f54c41a18ee1070249aafd9c81830699b360d0b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154) 
   
   <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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -215,82 +226,126 @@ protected Pair<CompletableFuture, ExecutorService> startService() {
       }, executor), executor);
     }
 
+    /**
+     * Follows the same execution methodology of HoodieFlinkCompactor, where only one clustering job
+     * is allowed to be executed at any point in time.
+     * <p>
+     * If there is an inflight clustering job, it will be rolled back and re-attempted.
+     * <p>
+     * A clustering plan will be generated if `schedule` is true.
+     *
+     * @throws Exception
+     * @see HoodieFlinkCompactor
+     */
     private void cluster() throws Exception {
       table.getMetaClient().reloadActiveTimeline();
 
-      // judges whether there are operations
-      // to compute the clustering instant time and exec clustering.
       if (cfg.schedule) {
+        // create a clustering plan on the timeline
         ClusteringUtil.validateClusteringScheduling(conf);
-        String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();
-        boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());
+
+        String clusteringInstantTime = cfg.clusteringInstantTime != null ? cfg.clusteringInstantTime
+            : HoodieActiveTimeline.createNewInstantTime();
+
+        LOG.info("Creating a clustering plan for instant [" + clusteringInstantTime + "]");
+        boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime,
+            Option.empty());
         if (!scheduled) {
           // do nothing.
           LOG.info("No clustering plan for this job");
+          executeDummyPipeline();
           return;
         }
         table.getMetaClient().reloadActiveTimeline();
       }
 
       // fetch the instant based on the configured execution sequence
-      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()).stream()
-          .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).collect(Collectors.toList());
+      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(
+          table.getMetaClient());
       if (instants.isEmpty()) {
         // do nothing.
-        LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option");
+        LOG.info(

Review Comment:
   Unnecessary change



-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -69,13 +83,14 @@ public class FlinkClusteringConfig extends Configuration {
       required = false)
   public Integer archiveMaxCommits = 30;
 
-  @Parameter(names = {"--schedule", "-sc"}, description = "Not recommended. Schedule the clustering plan in this job.\n"
-      + "There is a risk of losing data when scheduling clustering outside the writer job.\n"
-      + "Scheduling clustering in the writer job and only let this job do the clustering execution is recommended.\n"
-      + "Default is true", required = false)
-  public Boolean schedule = true;
+  @Parameter(names = {"--schedule", "-sc"}, description = "Schedule the clustering plan in this job.\n"
+      + "Default is false", required = false)
+  public Boolean schedule = false;
+
+  @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time")
+  public String clusteringInstantTime = null;

Review Comment:
   From `HoodieClusteringJob.java`
   ```
   @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time, only used when set --mode execute. "
           + "If the instant time is not provided with --mode execute, "
           + "the earliest scheduled clustering instant time is used by default. "
           + "When set \"--mode scheduleAndExecute\" this instant-time will be ignored.")
       public String clusteringInstantTime = null;
   ```
   
   Should we standardise the parameter? Given that the Spark parameter is using `--instant-time`, we should ensure that both of them are the same to avoid confusion.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a53950c478652bf65a959fa1f6f3930a84635d22 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086) 
   
   <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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   How do you submit the job, it weird what the no `execute()` exception throws because we return early in the executor worker thread.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -179,10 +184,16 @@ public AsyncClusteringService(FlinkClusteringConfig cfg, Configuration conf, Str
       conf.setString(FlinkOptions.TABLE_TYPE, metaClient.getTableConfig().getTableType().name());
 
       // set record key field
-      conf.setString(FlinkOptions.RECORD_KEY_FIELD, metaClient.getTableConfig().getRecordKeyFieldProp());
+      conf.setString(FlinkOptions.RECORD_KEY_FIELD,
+          metaClient.getTableConfig().getRecordKeyFieldProp());

Review Comment:
   Unnecessary change.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * e8100c4d856971de8dd42ba239a4f029d6ce676e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115) 
   
   <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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   Updated the code for both `compactor` and `clusteringjob` to include a catch block like this:
   
   ```java
   try {
     compactionScheduleService.compact();
   } catch (ApplicationExecutionException aee) {
     if (aee.getMessage().contains(NO_EXECUTE_CALL_KEYWORD)) {
       LOG.info("Compaction is not performed");
     } else {
       throw aee;
     }
   } catch (Exception e) {
     LOG.error("Got error running delta sync once. Shutting down", e);
     throw e;
   } finally {
     LOG.info("Shut down hoodie flink compactor");
   }
   ```
   
   Commit encompassing this change: e1f6bec7246b564266af0959ff448cb0aa2ef972



-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -62,6 +64,8 @@ public class HoodieFlinkClusteringJob {
 
   protected static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkClusteringJob.class);
 
+  private static final String NO_EXECUTE_CALL_KEYWORD = "The application contains no execute() calls";
+

Review Comment:
   Done



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154",
       "triggerID" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * f54c41a18ee1070249aafd9c81830699b360d0b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154) 
   * 38b7cfffeb455c6745b7d23445e48292567e7b9a UNKNOWN
   * d474c352d06287240f1b26c4bb31931eb216717c 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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154",
       "triggerID" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1768dc61360813b1240679c5f89d037c70135855",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11207",
       "triggerID" : "1768dc61360813b1240679c5f89d037c70135855",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3d8adcef64636820081e56b4876cf23a7efcbe53",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11210",
       "triggerID" : "3d8adcef64636820081e56b4876cf23a7efcbe53",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * f54c41a18ee1070249aafd9c81830699b360d0b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154) 
   * 38b7cfffeb455c6745b7d23445e48292567e7b9a UNKNOWN
   * d474c352d06287240f1b26c4bb31931eb216717c UNKNOWN
   * 1768dc61360813b1240679c5f89d037c70135855 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11207) 
   * 3d8adcef64636820081e56b4876cf23a7efcbe53 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11210) 
   
   <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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   While the exception throw does not affect anything (For most parts). 
   
   I would argue that it is more confusing for the it to throw an exception when there are no compaction/clustering plans to execute.
   
   If downstream applications are used to determine the status of the job, we would deem that this job has failed given that an exception is thrown (unless we explicitly scan the log to look for keywords to indicate that the failure is intended). 
   
   My goal here is to remove this such unintended errors to avoid operational confusion. In such a case, instead of adding an execution logic to avoid confusion, can we add an additional catch block to catch `ApplicationExecutionException`?



-- 
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 #6566: [HUDI-4766] Strengthen flink clustering job

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154",
       "triggerID" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1768dc61360813b1240679c5f89d037c70135855",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11207",
       "triggerID" : "1768dc61360813b1240679c5f89d037c70135855",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3d8adcef64636820081e56b4876cf23a7efcbe53",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11210",
       "triggerID" : "3d8adcef64636820081e56b4876cf23a7efcbe53",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * 38b7cfffeb455c6745b7d23445e48292567e7b9a UNKNOWN
   * d474c352d06287240f1b26c4bb31931eb216717c UNKNOWN
   * 3d8adcef64636820081e56b4876cf23a7efcbe53 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11210) 
   
   <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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   @danny0405 Error message will not happen in service mode. 
   
   When running under service mode, the job will wait for a plan to be created.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   How was the job submitted with such error stack trace ?



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   > can we add an additional catch block to catch ApplicationExecutionException
   
   We can if we can analyze the `no execute` error message.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154",
       "triggerID" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1768dc61360813b1240679c5f89d037c70135855",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11207",
       "triggerID" : "1768dc61360813b1240679c5f89d037c70135855",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3d8adcef64636820081e56b4876cf23a7efcbe53",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3d8adcef64636820081e56b4876cf23a7efcbe53",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * f54c41a18ee1070249aafd9c81830699b360d0b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154) 
   * 38b7cfffeb455c6745b7d23445e48292567e7b9a UNKNOWN
   * d474c352d06287240f1b26c4bb31931eb216717c UNKNOWN
   * 1768dc61360813b1240679c5f89d037c70135855 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11207) 
   * 3d8adcef64636820081e56b4876cf23a7efcbe53 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] danny0405 commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -116,14 +127,42 @@ public class FlinkClusteringConfig extends Configuration {
       description = "Min clustering interval of async clustering service, default 10 minutes")
   public Integer minClusteringIntervalSeconds = 600;
 
+  @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
+          + "(using the CLI parameter \"--props\") can also be passed command line using this parameter.")
+  public List<String> configs = new ArrayList<>();

Review Comment:
   `be passed command line` -> `be passed through command line`



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -211,28 +214,35 @@ private void compact() throws Exception {
 
       // checks the compaction plan and do compaction.
       if (cfg.schedule) {
-        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient);
+        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(
+            metaClient);
         if (compactionInstantTimeOption.isPresent()) {
           boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
           if (!scheduled) {
             // do nothing.
             LOG.info("No compaction plan for this job ");
+            executeDummyPipeline();
             return;
           }
           table.getMetaClient().reloadActiveTimeline();
         }
       }
 
       // fetch the instant based on the configured execution sequence
-      HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
-      List<HoodieInstant> requested = CompactionPlanStrategies.getStrategy(cfg).select(pendingCompactionTimeline);
+      HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline()
+          .filterPendingCompactionTimeline();
+      List<HoodieInstant> requested = CompactionPlanStrategies.getStrategy(cfg)
+          .select(pendingCompactionTimeline);
       if (requested.isEmpty()) {
         // do nothing.
-        LOG.info("No compaction plan scheduled, turns on the compaction plan schedule with --schedule option");
+        LOG.info(
+            "No compaction plan scheduled, turns on the compaction plan schedule with --schedule option");

Review Comment:
   Revert all the unnecessary change.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -211,28 +214,35 @@ private void compact() throws Exception {
 
       // checks the compaction plan and do compaction.
       if (cfg.schedule) {
-        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient);
+        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(
+            metaClient);
         if (compactionInstantTimeOption.isPresent()) {
           boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
           if (!scheduled) {
             // do nothing.
             LOG.info("No compaction plan for this job ");
+            executeDummyPipeline();
             return;
           }
           table.getMetaClient().reloadActiveTimeline();
         }
       }
 
       // fetch the instant based on the configured execution sequence
-      HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
-      List<HoodieInstant> requested = CompactionPlanStrategies.getStrategy(cfg).select(pendingCompactionTimeline);
+      HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline()
+          .filterPendingCompactionTimeline();
+      List<HoodieInstant> requested = CompactionPlanStrategies.getStrategy(cfg)
+          .select(pendingCompactionTimeline);
       if (requested.isEmpty()) {
         // do nothing.
-        LOG.info("No compaction plan scheduled, turns on the compaction plan schedule with --schedule option");
+        LOG.info(
+            "No compaction plan scheduled, turns on the compaction plan schedule with --schedule option");
+        executeDummyPipeline();
         return;
       }
 
-      List<String> compactionInstantTimes = requested.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
+      List<String> compactionInstantTimes = requested.stream().map(HoodieInstant::getTimestamp)
+          .collect(Collectors.toList());

Review Comment:
   Revert all the unnecessary change.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -301,7 +332,8 @@ private void cluster() throws Exception {
       long ckpTimeout = env.getCheckpointConfig().getCheckpointTimeout();
       conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout);
 
-      DataStream<ClusteringCommitEvent> dataStream = env.addSource(new ClusteringPlanSourceFunction(clusteringInstant.getTimestamp(), clusteringPlan))
+      DataStream<ClusteringCommitEvent> dataStream = env.addSource(
+              new ClusteringPlanSourceFunction(clusteringInstant.getTimestamp(), clusteringPlan))

Review Comment:
   Revert all the unnecessary change.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -158,7 +160,8 @@ public static class AsyncCompactionService extends HoodieAsyncTableService {
      */
     private final ExecutorService executor;
 
-    public AsyncCompactionService(FlinkCompactionConfig cfg, Configuration conf, StreamExecutionEnvironment env) throws Exception {
+    public AsyncCompactionService(FlinkCompactionConfig cfg, Configuration conf,
+        StreamExecutionEnvironment env) throws Exception {

Review Comment:
   Revert all the unnecessary change.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -211,28 +214,35 @@ private void compact() throws Exception {
 
       // checks the compaction plan and do compaction.
       if (cfg.schedule) {
-        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient);
+        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(
+            metaClient);

Review Comment:
   Revert all the unnecessary change.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * 257a2f2acf08448c082c89510cd731b4d8f1b877 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130) 
   
   <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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * 4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088) 
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * a9c0e105f4f90f8403649a3a01bddaf3db23ed03 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105) 
   
   <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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * 257a2f2acf08448c082c89510cd731b4d8f1b877 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130) 
   * 1709f71ae9494da4d7ca6b9c62ac97cd11dd8046 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146) 
   
   <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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   <img width="1406" alt="image" src="https://user-images.githubusercontent.com/6312314/188038599-9df06074-0ad3-488b-a07b-5076476a3458.png">
   



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a53950c478652bf65a959fa1f6f3930a84635d22 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086) 
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 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] danny0405 commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -259,26 +271,31 @@ private void compact() throws Exception {
       if (compactionPlans.isEmpty()) {
         // No compaction plan, do nothing and return.
         LOG.info("No compaction plan for instant " + String.join(",", compactionInstantTimes));
+        executeDummyPipeline();
         return;
       }
 
-      List<HoodieInstant> instants = compactionInstantTimes.stream().map(HoodieTimeline::getCompactionRequestedInstant).collect(Collectors.toList());
+      List<HoodieInstant> instants = compactionInstantTimes.stream()
+          .map(HoodieTimeline::getCompactionRequestedInstant).collect(Collectors.toList());
       for (HoodieInstant instant : instants) {
         if (!pendingCompactionTimeline.containsInstant(instant)) {
           // this means that the compaction plan was written to auxiliary path(.tmp)
           // but not the meta path(.hoodie), this usually happens when the job crush
           // exceptionally.
           // clean the compaction plan in auxiliary path and cancels the compaction.
-          LOG.warn("The compaction plan was fetched through the auxiliary path(.tmp) but not the meta path(.hoodie).\n"
-              + "Clean the compaction plan in auxiliary path and cancels the compaction");
+          LOG.warn(
+              "The compaction plan was fetched through the auxiliary path(.tmp) but not the meta path(.hoodie).\n"
+                  + "Clean the compaction plan in auxiliary path and cancels the compaction");
           CompactionUtil.cleanInstant(table.getMetaClient(), instant);

Review Comment:
   Revert all the unnecessary change.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -259,26 +271,31 @@ private void compact() throws Exception {
       if (compactionPlans.isEmpty()) {
         // No compaction plan, do nothing and return.
         LOG.info("No compaction plan for instant " + String.join(",", compactionInstantTimes));
+        executeDummyPipeline();
         return;
       }
 
-      List<HoodieInstant> instants = compactionInstantTimes.stream().map(HoodieTimeline::getCompactionRequestedInstant).collect(Collectors.toList());
+      List<HoodieInstant> instants = compactionInstantTimes.stream()
+          .map(HoodieTimeline::getCompactionRequestedInstant).collect(Collectors.toList());
       for (HoodieInstant instant : instants) {
         if (!pendingCompactionTimeline.containsInstant(instant)) {
           // this means that the compaction plan was written to auxiliary path(.tmp)
           // but not the meta path(.hoodie), this usually happens when the job crush
           // exceptionally.
           // clean the compaction plan in auxiliary path and cancels the compaction.
-          LOG.warn("The compaction plan was fetched through the auxiliary path(.tmp) but not the meta path(.hoodie).\n"
-              + "Clean the compaction plan in auxiliary path and cancels the compaction");
+          LOG.warn(
+              "The compaction plan was fetched through the auxiliary path(.tmp) but not the meta path(.hoodie).\n"
+                  + "Clean the compaction plan in auxiliary path and cancels the compaction");
           CompactionUtil.cleanInstant(table.getMetaClient(), instant);
+          executeDummyPipeline();
           return;
         }
       }
 
       // get compactionParallelism.
       int compactionParallelism = conf.getInteger(FlinkOptions.COMPACTION_TASKS) == -1
-          ? Math.toIntExact(compactionPlans.stream().mapToLong(pair -> pair.getRight().getOperations().size()).sum())
+          ? Math.toIntExact(
+          compactionPlans.stream().mapToLong(pair -> pair.getRight().getOperations().size()).sum())

Review Comment:
   Revert all the unnecessary change.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -259,26 +271,31 @@ private void compact() throws Exception {
       if (compactionPlans.isEmpty()) {
         // No compaction plan, do nothing and return.
         LOG.info("No compaction plan for instant " + String.join(",", compactionInstantTimes));
+        executeDummyPipeline();
         return;
       }
 
-      List<HoodieInstant> instants = compactionInstantTimes.stream().map(HoodieTimeline::getCompactionRequestedInstant).collect(Collectors.toList());
+      List<HoodieInstant> instants = compactionInstantTimes.stream()
+          .map(HoodieTimeline::getCompactionRequestedInstant).collect(Collectors.toList());

Review Comment:
   Revert all the unnecessary change.



-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   How do we do that? 
   
   Do we call `shutDown(false)` to achieve this?



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -211,28 +214,35 @@ private void compact() throws Exception {
 
       // checks the compaction plan and do compaction.
       if (cfg.schedule) {
-        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient);
+        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(
+            metaClient);
         if (compactionInstantTimeOption.isPresent()) {
           boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
           if (!scheduled) {
             // do nothing.
             LOG.info("No compaction plan for this job ");
+            executeDummyPipeline();
             return;
           }
           table.getMetaClient().reloadActiveTimeline();
         }
       }
 
       // fetch the instant based on the configured execution sequence
-      HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
-      List<HoodieInstant> requested = CompactionPlanStrategies.getStrategy(cfg).select(pendingCompactionTimeline);
+      HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline()
+          .filterPendingCompactionTimeline();
+      List<HoodieInstant> requested = CompactionPlanStrategies.getStrategy(cfg)

Review Comment:
   Revert all the unnecessary change.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -116,14 +131,57 @@ public class FlinkClusteringConfig extends Configuration {
       description = "Min clustering interval of async clustering service, default 10 minutes")
   public Integer minClusteringIntervalSeconds = 600;
 
+  @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
+          + "(using the CLI parameter \"--props\") can also be passed command line using this parameter.")
+  public List<String> configs = new ArrayList<>();
+
+  @Parameter(names = {"--props"}, description = "Path to properties file on localfs or dfs, with configurations for "
+          + "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are "
+          + "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer"
+          + "to individual classes, for supported properties.")
+  public String propsFilePath = "";
+
+  /**
+   * Read config from properties file (`--props` option) and cmd line (`--hoodie-conf` option).
+   */
+  public static DFSPropertiesConfiguration readConfig(org.apache.hadoop.conf.Configuration hadoopConfig, Path cfgPath, List<String> overriddenProps) {
+    DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(hadoopConfig, cfgPath);
+    try {
+      if (!overriddenProps.isEmpty()) {
+        conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps))));
+      }
+    } catch (IOException ioe) {
+      throw new HoodieIOException("Unexpected error adding config overrides", ioe);

Review Comment:
   Can the method in `UtilHelpers` be reused ?



-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   Hmmm, CMIIW, `execute()` is only triggered at the end of the `cluster()` function.
   
   ```java
   env.execute("flink_hudi_clustering_" + clusteringInstant.getTimestamp());
   ```
   
   If `cluster()` function terminates before reaching the end, the `execute()` function will not be called.
   
   Hence, there will be a `no execute() exception` as shown in the image. 
   
   Jobs are submitted via Flink web portal using the hoodie flink bundle jar. 



-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   Application is submitted and executed in a Flink Job Cluster.
   
   The command that is used to start the job:
   ```shell
   /bin/flink run-application \
   	-t yarn-application \
   	-p 4 \
   	-c org.apache.hudi.sink.compact.HoodieFlinkCompactor \
   	hdfs://hudi-flink.jar \
   	--path hdfs://hudi_mor_compaction_test/ \
   	--compaction-tasks 4
   ```



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154",
       "triggerID" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * 1709f71ae9494da4d7ca6b9c62ac97cd11dd8046 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146) 
   * f54c41a18ee1070249aafd9c81830699b360d0b9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154) 
   
   <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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -62,6 +64,8 @@ public class HoodieFlinkClusteringJob {
 
   protected static final Logger LOG = LoggerFactory.getLogger(HoodieFlinkClusteringJob.class);
 
+  private static final String NO_EXECUTE_CALL_KEYWORD = "The application contains no execute() calls";
+

Review Comment:
   We can simplified the keyword as `no execute` instead.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -247,9 +257,11 @@ private void compact() throws Exception {
       List<Pair<String, HoodieCompactionPlan>> compactionPlans = compactionInstantTimes.stream()
           .map(timestamp -> {
             try {
-              return Pair.of(timestamp, CompactionUtils.getCompactionPlan(table.getMetaClient(), timestamp));
+              return Pair.of(timestamp,

Review Comment:
   Revert all the unnecessary change.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -247,9 +257,11 @@ private void compact() throws Exception {
       List<Pair<String, HoodieCompactionPlan>> compactionPlans = compactionInstantTimes.stream()
           .map(timestamp -> {
             try {
-              return Pair.of(timestamp, CompactionUtils.getCompactionPlan(table.getMetaClient(), timestamp));
+              return Pair.of(timestamp,
+                  CompactionUtils.getCompactionPlan(table.getMetaClient(), timestamp));
             } catch (IOException e) {
-              throw new HoodieException("Get compaction plan at instant " + timestamp + " error", e);
+              throw new HoodieException("Get compaction plan at instant " + timestamp + " error",
+                  e);

Review Comment:
   Revert all the unnecessary change.



-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -215,82 +226,126 @@ protected Pair<CompletableFuture, ExecutorService> startService() {
       }, executor), executor);
     }
 
+    /**
+     * Follows the same execution methodology of HoodieFlinkCompactor, where only one clustering job
+     * is allowed to be executed at any point in time.
+     * <p>
+     * If there is an inflight clustering job, it will be rolled back and re-attempted.
+     * <p>
+     * A clustering plan will be generated if `schedule` is true.
+     *
+     * @throws Exception
+     * @see HoodieFlinkCompactor
+     */
     private void cluster() throws Exception {
       table.getMetaClient().reloadActiveTimeline();
 
-      // judges whether there are operations
-      // to compute the clustering instant time and exec clustering.
       if (cfg.schedule) {
+        // create a clustering plan on the timeline
         ClusteringUtil.validateClusteringScheduling(conf);
-        String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();
-        boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());
+
+        String clusteringInstantTime = cfg.clusteringInstantTime != null ? cfg.clusteringInstantTime
+            : HoodieActiveTimeline.createNewInstantTime();
+
+        LOG.info("Creating a clustering plan for instant [" + clusteringInstantTime + "]");
+        boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime,
+            Option.empty());
         if (!scheduled) {
           // do nothing.
           LOG.info("No clustering plan for this job");
+          executeDummyPipeline();
           return;
         }
         table.getMetaClient().reloadActiveTimeline();
       }
 
       // fetch the instant based on the configured execution sequence
-      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()).stream()
-          .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).collect(Collectors.toList());
+      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(
+          table.getMetaClient());
       if (instants.isEmpty()) {
         // do nothing.
-        LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option");
+        LOG.info(
+            "No clustering plan scheduled, turns on the clustering plan schedule with --schedule option");
+        executeDummyPipeline();
         return;
       }
 
-      HoodieInstant clusteringInstant = CompactionUtil.isLIFO(cfg.clusteringSeq) ? instants.get(instants.size() - 1) : instants.get(0);
+      HoodieInstant reqClusteringInstant;
+      if (cfg.clusteringInstantTime != null) {
+        List<HoodieInstant> reqHoodieInstant = instants
+            .stream()
+            .filter(i -> i.getTimestamp().equals(cfg.clusteringInstantTime))

Review Comment:
   Using this instead:
   
   ```java
   reqClusteringInstant = instants.stream()
               .filter(i -> i.getTimestamp().equals(cfg.clusteringInstantTime))
               .findFirst()
               .orElseThrow(() -> new HoodieException("Clustering instant [" + cfg.clusteringInstantTime + "] not found"));
   ```



-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -215,37 +217,63 @@ protected Pair<CompletableFuture, ExecutorService> startService() {
       }, executor), executor);
     }
 
+    /**
+     * Follows the same execution methodology of HoodieFlinkCompactor, where only one clustering job is allowed to be
+     * executed at any point in time.
+     * <p>
+     * If there is an inflight clustering job, it will be rolled back and re-attempted.
+     * <p>
+     * A clustering plan will be generated if `schedule` is true.
+     *
+     * @throws Exception
+     * @see HoodieFlinkCompactor
+     */
     private void cluster() throws Exception {
       table.getMetaClient().reloadActiveTimeline();
 
-      // judges whether there are operations
-      // to compute the clustering instant time and exec clustering.
       if (cfg.schedule) {
+        // create a clustering plan on the timeline
         ClusteringUtil.validateClusteringScheduling(conf);
-        String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();
+
+        String clusteringInstantTime = cfg.clusteringInstantTime != null ? cfg.clusteringInstantTime
+            : HoodieActiveTimeline.createNewInstantTime();
+
+        LOG.info("Creating a clustering plan for instant [" + clusteringInstantTime + "]");
         boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());
         if (!scheduled) {
           // do nothing.
           LOG.info("No clustering plan for this job");
+          executeDummyPipeline();
           return;
         }
         table.getMetaClient().reloadActiveTimeline();
       }
 
       // fetch the instant based on the configured execution sequence
-      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()).stream()
-          .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).collect(Collectors.toList());
+      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient());
       if (instants.isEmpty()) {
         // do nothing.
         LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option");
+        executeDummyPipeline();
         return;
       }
 
-      HoodieInstant clusteringInstant = CompactionUtil.isLIFO(cfg.clusteringSeq) ? instants.get(instants.size() - 1) : instants.get(0);
+      HoodieInstant reqClusteringInstant;
+      if (cfg.clusteringInstantTime != null) {

Review Comment:
   Fixed



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a53950c478652bf65a959fa1f6f3930a84635d22 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086) 
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * 4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088) 
   
   <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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * a9c0e105f4f90f8403649a3a01bddaf3db23ed03 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105) 
   * e8100c4d856971de8dd42ba239a4f029d6ce676e 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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a53950c478652bf65a959fa1f6f3930a84635d22 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086) 
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * 4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088) 
   
   <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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -116,14 +131,57 @@ public class FlinkClusteringConfig extends Configuration {
       description = "Min clustering interval of async clustering service, default 10 minutes")
   public Integer minClusteringIntervalSeconds = 600;
 
+  @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
+          + "(using the CLI parameter \"--props\") can also be passed command line using this parameter.")
+  public List<String> configs = new ArrayList<>();
+
+  @Parameter(names = {"--props"}, description = "Path to properties file on localfs or dfs, with configurations for "
+          + "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are "
+          + "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer"
+          + "to individual classes, for supported properties.")
+  public String propsFilePath = "";
+
+  /**
+   * Read config from properties file (`--props` option) and cmd line (`--hoodie-conf` option).
+   */
+  public static DFSPropertiesConfiguration readConfig(org.apache.hadoop.conf.Configuration hadoopConfig, Path cfgPath, List<String> overriddenProps) {
+    DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(hadoopConfig, cfgPath);
+    try {
+      if (!overriddenProps.isEmpty()) {
+        conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps))));
+      }
+    } catch (IOException ioe) {
+      throw new HoodieIOException("Unexpected error adding config overrides", ioe);

Review Comment:
   Can the method in `StreamerUtil.readConfig` be reused ?



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -215,35 +217,61 @@ protected Pair<CompletableFuture, ExecutorService> startService() {
       }, executor), executor);
     }
 
+    /**
+     * Follows the same execution methodology of HoodieFlinkCompactor, where only one clustering job is allowed to be
+     * executed at any point in time.
+     * <p>
+     * If there is an inflight clustering job, it will be rolled back and re-attempted.
+     * <p>
+     * A clustering plan will be generated if `schedule` is true.
+     *
+     * @throws Exception
+     * @see HoodieFlinkCompactor
+     */
     private void cluster() throws Exception {
       table.getMetaClient().reloadActiveTimeline();
 
-      // judges whether there are operations
-      // to compute the clustering instant time and exec clustering.
       if (cfg.schedule) {
+        // create a clustering plan on the timeline
         ClusteringUtil.validateClusteringScheduling(conf);
-        String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();
+
+        String clusteringInstantTime = cfg.clusteringInstantTime != null ? cfg.clusteringInstantTime
+            : HoodieActiveTimeline.createNewInstantTime();
+
+        LOG.info("Creating a clustering plan for instant [" + clusteringInstantTime + "]");
         boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());
         if (!scheduled) {
           // do nothing.
           LOG.info("No clustering plan for this job");
+          executeDummyPipeline();
           return;
         }
         table.getMetaClient().reloadActiveTimeline();
       }
 
       // fetch the instant based on the configured execution sequence
-      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()).stream()
-          .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).collect(Collectors.toList());
+      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient());
       if (instants.isEmpty()) {
         // do nothing.
         LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option");
+        executeDummyPipeline();
         return;
       }
 
-      HoodieInstant clusteringInstant = CompactionUtil.isLIFO(cfg.clusteringSeq) ? instants.get(instants.size() - 1) : instants.get(0);
+      HoodieInstant clusteringInstant;
+      if (cfg.clusteringInstantTime != null) {

Review Comment:
   `HoodieInstant clusteringInstant;` -> `final HoodieInstant clusteringInstant;`



-- 
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 #6566: [HUDI-4766] Strengthen flink clustering job

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154",
       "triggerID" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1768dc61360813b1240679c5f89d037c70135855",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11207",
       "triggerID" : "1768dc61360813b1240679c5f89d037c70135855",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3d8adcef64636820081e56b4876cf23a7efcbe53",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11210",
       "triggerID" : "3d8adcef64636820081e56b4876cf23a7efcbe53",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * 38b7cfffeb455c6745b7d23445e48292567e7b9a UNKNOWN
   * d474c352d06287240f1b26c4bb31931eb216717c UNKNOWN
   * 1768dc61360813b1240679c5f89d037c70135855 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11207) 
   * 3d8adcef64636820081e56b4876cf23a7efcbe53 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11210) 
   
   <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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   Updated the code for both `compactor` and `clusteringjob` to include a catch block like this:
   
   ```java
   try {
     compactionScheduleService.compact();
   } catch (ApplicationExecutionException aee) {
     if (aee.getMessage().contains(NO_EXECUTE_CALL_KEYWORD)) {
       LOG.info("Compaction is not performed");
     } else {
       throw aee;
     }
   } catch (Exception e) {
     LOG.error("Got error running delta sync once. Shutting down", e);
     throw e;
   } finally {
     LOG.info("Shut down hoodie flink compactor");
   }
   ```



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154",
       "triggerID" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * f54c41a18ee1070249aafd9c81830699b360d0b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154) 
   * 38b7cfffeb455c6745b7d23445e48292567e7b9a 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] danny0405 merged pull request #6566: [HUDI-4766] Strengthen flink clustering job

Posted by GitBox <gi...@apache.org>.
danny0405 merged PR #6566:
URL: https://github.com/apache/hudi/pull/6566


-- 
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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -69,13 +83,14 @@ public class FlinkClusteringConfig extends Configuration {
       required = false)
   public Integer archiveMaxCommits = 30;
 
-  @Parameter(names = {"--schedule", "-sc"}, description = "Not recommended. Schedule the clustering plan in this job.\n"
-      + "There is a risk of losing data when scheduling clustering outside the writer job.\n"
-      + "Scheduling clustering in the writer job and only let this job do the clustering execution is recommended.\n"
-      + "Default is true", required = false)
-  public Boolean schedule = true;
+  @Parameter(names = {"--schedule", "-sc"}, description = "Schedule the clustering plan in this job.\n"
+      + "Default is false", required = false)
+  public Boolean schedule = false;
+
+  @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time")
+  public String clusteringInstantTime = null;

Review Comment:
   In the scenario where users are running a scheduled execution running schedule & execute w/ LIFO instant selection + probability of a job failure. 
   
   Users can re-execute a failed clustering operation by specifying `--instant-time`.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java:
##########
@@ -211,28 +214,36 @@ private void compact() throws Exception {
 
       // checks the compaction plan and do compaction.
       if (cfg.schedule) {
-        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient);
+        Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(
+            metaClient);
         if (compactionInstantTimeOption.isPresent()) {
-          boolean scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
+          boolean scheduled = writeClient.scheduleCompactionAtInstant(
+              compactionInstantTimeOption.get(), Option.empty());

Review Comment:
   Can we revert these unnecessary changes ?



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * a9c0e105f4f90f8403649a3a01bddaf3db23ed03 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105) 
   * e8100c4d856971de8dd42ba239a4f029d6ce676e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115) 
   
   <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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -259,15 +287,18 @@ private void cluster() throws Exception {
       if (!clusteringPlanOption.isPresent()) {
         // do nothing.
         LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option");
+        executeDummyPipeline();
         return;
       }
 
+      clusteringInstant = clusteringPlanOption.get().getLeft();
       HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight();

Review Comment:
   Is there any special reason we need to overwrite the `clusteringInstant` from the `clusteringPlanOption` ?



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * 257a2f2acf08448c082c89510cd731b4d8f1b877 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130) 
   * 1709f71ae9494da4d7ca6b9c62ac97cd11dd8046 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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * 1709f71ae9494da4d7ca6b9c62ac97cd11dd8046 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146) 
   
   <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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -215,35 +217,61 @@ protected Pair<CompletableFuture, ExecutorService> startService() {
       }, executor), executor);
     }
 
+    /**
+     * Follows the same execution methodology of HoodieFlinkCompactor, where only one clustering job is allowed to be
+     * executed at any point in time.
+     * <p>
+     * If there is an inflight clustering job, it will be rolled back and re-attempted.
+     * <p>
+     * A clustering plan will be generated if `schedule` is true.
+     *
+     * @throws Exception
+     * @see HoodieFlinkCompactor
+     */
     private void cluster() throws Exception {
       table.getMetaClient().reloadActiveTimeline();
 
-      // judges whether there are operations
-      // to compute the clustering instant time and exec clustering.
       if (cfg.schedule) {
+        // create a clustering plan on the timeline
         ClusteringUtil.validateClusteringScheduling(conf);
-        String clusteringInstantTime = HoodieActiveTimeline.createNewInstantTime();
+
+        String clusteringInstantTime = cfg.clusteringInstantTime != null ? cfg.clusteringInstantTime
+            : HoodieActiveTimeline.createNewInstantTime();
+
+        LOG.info("Creating a clustering plan for instant [" + clusteringInstantTime + "]");
         boolean scheduled = writeClient.scheduleClusteringAtInstant(clusteringInstantTime, Option.empty());
         if (!scheduled) {
           // do nothing.
           LOG.info("No clustering plan for this job");
+          executeDummyPipeline();
           return;
         }
         table.getMetaClient().reloadActiveTimeline();
       }
 
       // fetch the instant based on the configured execution sequence
-      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()).stream()
-          .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).collect(Collectors.toList());
+      List<HoodieInstant> instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient());
       if (instants.isEmpty()) {
         // do nothing.
         LOG.info("No clustering plan scheduled, turns on the clustering plan schedule with --schedule option");
+        executeDummyPipeline();
         return;
       }
 
-      HoodieInstant clusteringInstant = CompactionUtil.isLIFO(cfg.clusteringSeq) ? instants.get(instants.size() - 1) : instants.get(0);
+      HoodieInstant clusteringInstant;
+      if (cfg.clusteringInstantTime != null) {

Review Comment:
   Change added.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -116,14 +127,42 @@ public class FlinkClusteringConfig extends Configuration {
       description = "Min clustering interval of async clustering service, default 10 minutes")
   public Integer minClusteringIntervalSeconds = 600;
 
+  @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
+          + "(using the CLI parameter \"--props\") can also be passed command line using this parameter.")
+  public List<String> configs = new ArrayList<>();

Review Comment:
   Change 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.

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

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


[GitHub] [hudi] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   I have not tested it on service mode before since our use case doesn't call for it. 
   
   Is there a better way around this?



-- 
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] voonhous commented on pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   @danny0405 Added the changes that you suggested. Thank you.


-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * e8100c4d856971de8dd42ba239a4f029d6ce676e Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115) 
   * 257a2f2acf08448c082c89510cd731b4d8f1b877 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] voonhous commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/FlinkClusteringConfig.java:
##########
@@ -116,14 +131,57 @@ public class FlinkClusteringConfig extends Configuration {
       description = "Min clustering interval of async clustering service, default 10 minutes")
   public Integer minClusteringIntervalSeconds = 600;
 
+  @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
+          + "(using the CLI parameter \"--props\") can also be passed command line using this parameter.")
+  public List<String> configs = new ArrayList<>();
+
+  @Parameter(names = {"--props"}, description = "Path to properties file on localfs or dfs, with configurations for "
+          + "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are "
+          + "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer"
+          + "to individual classes, for supported properties.")
+  public String propsFilePath = "";
+
+  /**
+   * Read config from properties file (`--props` option) and cmd line (`--hoodie-conf` option).
+   */
+  public static DFSPropertiesConfiguration readConfig(org.apache.hadoop.conf.Configuration hadoopConfig, Path cfgPath, List<String> overriddenProps) {
+    DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(hadoopConfig, cfgPath);
+    try {
+      if (!overriddenProps.isEmpty()) {
+        conf.addPropsFromStream(new BufferedReader(new StringReader(String.join("\n", overriddenProps))));
+      }
+    } catch (IOException ioe) {
+      throw new HoodieIOException("Unexpected error adding config overrides", ioe);

Review Comment:
   Yes, will make the change.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -179,10 +184,16 @@ public AsyncClusteringService(FlinkClusteringConfig cfg, Configuration conf, Str
       conf.setString(FlinkOptions.TABLE_TYPE, metaClient.getTableConfig().getTableType().name());
 
       // set record key field
-      conf.setString(FlinkOptions.RECORD_KEY_FIELD, metaClient.getTableConfig().getRecordKeyFieldProp());
+      conf.setString(FlinkOptions.RECORD_KEY_FIELD,
+          metaClient.getTableConfig().getRecordKeyFieldProp());
 
       // set partition field
-      conf.setString(FlinkOptions.PARTITION_PATH_FIELD, metaClient.getTableConfig().getPartitionFieldProp());
+      conf.setString(FlinkOptions.PARTITION_PATH_FIELD,

Review Comment:
   Unnecessary change.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * 4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088) 
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * a9c0e105f4f90f8403649a3a01bddaf3db23ed03 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] danny0405 commented on a diff in pull request #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java:
##########
@@ -335,5 +391,17 @@ public void shutdownAsyncService(boolean error) {
     public void shutDown() {
       shutdownAsyncService(false);
     }
+
+    /**
+     * Execute a dummy pipeline to prevent "no execute() calls" exceptions from being thrown if
+     * clustering is not performed.
+     */

Review Comment:
   Let's remove the dummy execution logic first to avoid confusion (a successful execution that does nothing), an exception throws here does not affect anything right ? And the user can know that there is no compaction plan that needs to execute.



-- 
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 #6566: [HUDI-4766] Fix HoodieFlinkClusteringJob

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11086",
       "triggerID" : "a53950c478652bf65a959fa1f6f3930a84635d22",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b10c9d062f03c2c2675866c6f4bf6346dc03ea49",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11088",
       "triggerID" : "4d1ab92d3dbf11e934d07f8e6fb09524f1288f1b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a2dcd81f74603e88c4db895900d43eee6702a6da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c404647afc6d26bc0e69a7a8ef93f378b397bb96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11105",
       "triggerID" : "a9c0e105f4f90f8403649a3a01bddaf3db23ed03",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11115",
       "triggerID" : "e8100c4d856971de8dd42ba239a4f029d6ce676e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11130",
       "triggerID" : "257a2f2acf08448c082c89510cd731b4d8f1b877",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11146",
       "triggerID" : "1709f71ae9494da4d7ca6b9c62ac97cd11dd8046",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154",
       "triggerID" : "f54c41a18ee1070249aafd9c81830699b360d0b9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38b7cfffeb455c6745b7d23445e48292567e7b9a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d474c352d06287240f1b26c4bb31931eb216717c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1768dc61360813b1240679c5f89d037c70135855",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11207",
       "triggerID" : "1768dc61360813b1240679c5f89d037c70135855",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b10c9d062f03c2c2675866c6f4bf6346dc03ea49 UNKNOWN
   * a2dcd81f74603e88c4db895900d43eee6702a6da UNKNOWN
   * c404647afc6d26bc0e69a7a8ef93f378b397bb96 UNKNOWN
   * f54c41a18ee1070249aafd9c81830699b360d0b9 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11154) 
   * 38b7cfffeb455c6745b7d23445e48292567e7b9a UNKNOWN
   * d474c352d06287240f1b26c4bb31931eb216717c UNKNOWN
   * 1768dc61360813b1240679c5f89d037c70135855 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11207) 
   
   <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