You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "Leoyzen (via GitHub)" <gi...@apache.org> on 2023/02/02 03:13:25 UTC

[GitHub] [hudi] Leoyzen opened a new issue, #7823: [SUPPORT]No execution calls after rollback compaction while using offline flink compactor.

Leoyzen opened a new issue, #7823:
URL: https://github.com/apache/hudi/issues/7823

   **_Tips before filing an issue_**
   
   - Have you gone through our [FAQs](https://hudi.apache.org/learn/faq/)?
   
   - Join the mailing list to engage in conversations and get faster support at dev-subscribe@hudi.apache.org.
   
   - If you have triaged this as a bug, then file an [issue](https://issues.apache.org/jira/projects/HUDI/issues) directly.
   
   **Describe the problem you faced**
   
   While using offline flink compactor, "no execute() calls" exception occurs after rollback previous compaction inflight.
   
   
   
   **To Reproduce**
   
   Steps to reproduce the behavior:
   
   1. There are compaction plans scheduled, and some task unfinished(INFLIGHT).
   2. Launch HoodieFlinkCompactor.
   3. The error occurs
   
   **Expected behavior**
   
   After rollback compaction infight, the task should be execute.
   
   **Environment Description**
   
   * Hudi version :
   
   0.13.0-rc1
   
   * Spark version :
   
   * Hive version :
   3.1.2
   
   * Hadoop version :
   3.1.3
   
   * Storage (HDFS/S3/GCS..) :
   OSS
   
   * Running on Docker? (yes/no) :
   yes, HA Cluster.
   
   **Additional context**
   
   Add any other context about the problem here.
   
   **Stacktrace**
   
   ```LOG
   2023-02-01 22:53:12,742 WARN  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rollback finished without deleting inflight instant file. Instant=[==>20230201175930385__compaction__INFLIGHT]
   2023-02-01 22:53:12,743 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Checking for file exists ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225312338.rollback.inflight
   2023-02-01 22:53:12,764 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Create new file for toInstant ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225312338.rollback
   2023-02-01 22:53:12,765 INFO  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rollback of Commits [20230201175930385] is complete
   2023-02-01 22:53:12,772 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Deleting instant [==>20230201175930385__compaction__INFLIGHT]
   2023-02-01 22:53:12,787 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Removed instant [==>20230201175930385__compaction__INFLIGHT]
   2023-02-01 22:53:12,957 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[20230201225312338__rollback__COMPLETED]}
   2023-02-01 22:53:12,957 INFO  org.apache.hudi.client.RunsTableService                      [] - Rollback inflight compaction instant: [20230201175424452]
   2023-02-01 22:53:13,160 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20230201225312957__rollback__REQUESTED]}
   2023-02-01 22:53:13,160 INFO  org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor [] - Requesting Rollback with instant time [==>20230201225312957__rollback__REQUESTED]
   2023-02-01 22:53:13,322 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20230201225312957__rollback__REQUESTED]}
   2023-02-01 22:53:13,336 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Checking for file exists ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225312957.rollback.requested
   2023-02-01 22:53:13,360 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Create new file for toInstant ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225312957.rollback.inflight
   2023-02-01 22:53:13,360 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Rolling back instant [==>20230201175424452__compaction__INFLIGHT]
   2023-02-01 22:53:13,360 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Unpublished [==>20230201175424452__compaction__INFLIGHT]
   2023-02-01 22:53:13,360 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Time(in ms) taken to finish rollback 0
   2023-02-01 22:53:13,360 INFO  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rolled back inflight instant 20230201175424452
   2023-02-01 22:53:13,360 WARN  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rollback finished without deleting inflight instant file. Instant=[==>20230201175424452__compaction__INFLIGHT]
   2023-02-01 22:53:13,361 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Checking for file exists ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225312957.rollback.inflight
   2023-02-01 22:53:13,383 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Create new file for toInstant ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225312957.rollback
   2023-02-01 22:53:13,383 INFO  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rollback of Commits [20230201175424452] is complete
   2023-02-01 22:53:13,393 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Deleting instant [==>20230201175424452__compaction__INFLIGHT]
   2023-02-01 22:53:13,409 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Removed instant [==>20230201175424452__compaction__INFLIGHT]
   2023-02-01 22:53:13,596 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[20230201225312957__rollback__COMPLETED]}
   2023-02-01 22:53:13,596 INFO  org.apache.hudi.client.RunsTableService                      [] - Rollback inflight compaction instant: [20230201174918018]
   2023-02-01 22:53:13,812 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20230201225313596__rollback__REQUESTED]}
   2023-02-01 22:53:13,812 INFO  org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor [] - Requesting Rollback with instant time [==>20230201225313596__rollback__REQUESTED]
   2023-02-01 22:53:13,981 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20230201225313596__rollback__REQUESTED]}
   2023-02-01 22:53:13,994 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Checking for file exists ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225313596.rollback.requested
   2023-02-01 22:53:14,012 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Create new file for toInstant ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225313596.rollback.inflight
   2023-02-01 22:53:14,012 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Rolling back instant [==>20230201174918018__compaction__INFLIGHT]
   2023-02-01 22:53:14,013 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Unpublished [==>20230201174918018__compaction__INFLIGHT]
   2023-02-01 22:53:14,013 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Time(in ms) taken to finish rollback 1
   2023-02-01 22:53:14,013 INFO  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rolled back inflight instant 20230201174918018
   2023-02-01 22:53:14,013 WARN  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rollback finished without deleting inflight instant file. Instant=[==>20230201174918018__compaction__INFLIGHT]
   2023-02-01 22:53:14,013 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Checking for file exists ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225313596.rollback.inflight
   2023-02-01 22:53:14,031 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Create new file for toInstant ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225313596.rollback
   2023-02-01 22:53:14,032 INFO  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rollback of Commits [20230201174918018] is complete
   2023-02-01 22:53:14,039 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Deleting instant [==>20230201174918018__compaction__INFLIGHT]
   2023-02-01 22:53:14,055 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Removed instant [==>20230201174918018__compaction__INFLIGHT]
   2023-02-01 22:53:14,229 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[20230201225313596__rollback__COMPLETED]}
   2023-02-01 22:53:14,229 INFO  org.apache.hudi.client.RunsTableService                      [] - Rollback inflight compaction instant: [20230201174412438]
   2023-02-01 22:53:14,428 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20230201225314229__rollback__REQUESTED]}
   2023-02-01 22:53:14,428 INFO  org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor [] - Requesting Rollback with instant time [==>20230201225314229__rollback__REQUESTED]
   2023-02-01 22:53:14,593 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20230201225314229__rollback__REQUESTED]}
   2023-02-01 22:53:14,604 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Checking for file exists ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225314229.rollback.requested
   2023-02-01 22:53:14,621 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Create new file for toInstant ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225314229.rollback.inflight
   2023-02-01 22:53:14,622 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Rolling back instant [==>20230201174412438__compaction__INFLIGHT]
   2023-02-01 22:53:14,622 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Unpublished [==>20230201174412438__compaction__INFLIGHT]
   2023-02-01 22:53:14,622 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Time(in ms) taken to finish rollback 0
   2023-02-01 22:53:14,622 INFO  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rolled back inflight instant 20230201174412438
   2023-02-01 22:53:14,622 WARN  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rollback finished without deleting inflight instant file. Instant=[==>20230201174412438__compaction__INFLIGHT]
   2023-02-01 22:53:14,622 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Checking for file exists ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225314229.rollback.inflight
   2023-02-01 22:53:14,641 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Create new file for toInstant ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225314229.rollback
   2023-02-01 22:53:14,641 INFO  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rollback of Commits [20230201174412438] is complete
   2023-02-01 22:53:14,649 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Deleting instant [==>20230201174412438__compaction__INFLIGHT]
   2023-02-01 22:53:14,662 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Removed instant [==>20230201174412438__compaction__INFLIGHT]
   2023-02-01 22:53:14,831 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[20230201225314229__rollback__COMPLETED]}
   2023-02-01 22:53:14,831 INFO  org.apache.hudi.client.RunsTableService                      [] - Rollback inflight compaction instant: [20230201173905623]
   2023-02-01 22:53:15,026 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20230201225314831__rollback__REQUESTED]}
   2023-02-01 22:53:15,026 INFO  org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor [] - Requesting Rollback with instant time [==>20230201225314831__rollback__REQUESTED]
   2023-02-01 22:53:15,184 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[==>20230201225314831__rollback__REQUESTED]}
   2023-02-01 22:53:15,198 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Checking for file exists ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225314831.rollback.requested
   2023-02-01 22:53:15,217 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Create new file for toInstant ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225314831.rollback.inflight
   2023-02-01 22:53:15,217 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Rolling back instant [==>20230201173905623__compaction__INFLIGHT]
   2023-02-01 22:53:15,218 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Unpublished [==>20230201173905623__compaction__INFLIGHT]
   2023-02-01 22:53:15,218 INFO  org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor [] - Time(in ms) taken to finish rollback 1
   2023-02-01 22:53:15,218 INFO  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rolled back inflight instant 20230201173905623
   2023-02-01 22:53:15,218 WARN  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rollback finished without deleting inflight instant file. Instant=[==>20230201173905623__compaction__INFLIGHT]
   2023-02-01 22:53:15,218 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Checking for file exists ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225314831.rollback.inflight
   2023-02-01 22:53:15,237 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Create new file for toInstant ?oss://dengine-lake-zjk/cloudcode_prod/dwd_egc_adv_resp_intra/.hoodie/20230201225314831.rollback
   2023-02-01 22:53:15,237 INFO  org.apache.hudi.table.action.rollback.BaseRollbackActionExecutor [] - Rollback of Commits [20230201173905623] is complete
   2023-02-01 22:53:15,246 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Deleting instant [==>20230201173905623__compaction__INFLIGHT]
   2023-02-01 22:53:15,263 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Removed instant [==>20230201173905623__compaction__INFLIGHT]
   2023-02-01 22:53:15,430 INFO  org.apache.hudi.common.table.timeline.HoodieActiveTimeline   [] - Loaded instants upto : Option{val=[20230201225314831__rollback__COMPLETED]}
   2023-02-01 22:53:15,696 WARN  org.apache.hudi.client.RunsTableService                      [] - The compaction plan was fetched through the auxiliary path(.tmp) but not the meta path(.hoodie).
   Clean the compaction plan in auxiliary path and cancels the compaction
   2023-02-01 22:53:15,704 INFO  org.apache.hudi.sink.compact.HoodieFlinkCompactor            [] - Shut down hoodie flink compactor
   2023-02-01 22:53:15,706 WARN  org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap [] - Application failed unexpectedly: 
   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_102]
   	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308) ~[?:1.8.0_102]
   	at java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:943) ~[?:1.8.0_102]
   	at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:926) ~[?:1.8.0_102]
   	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) ~[?:1.8.0_102]
   	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977) ~[?:1.8.0_102]
   	at org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap.runApplicationEntryPoint(ApplicationDispatcherBootstrap.java:299) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap.lambda$runApplicationAsync$2(ApplicationDispatcherBootstrap.java:244) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_102]
   	at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[?:1.8.0_102]
   	at org.apache.flink.runtime.concurrent.akka.ActorSystemScheduledExecutorAdapter$ScheduledFutureTask.run(ActorSystemScheduledExecutorAdapter.java:171) ~[flink-rpc-akka_9ece02b9-c5ce-40c7-96a1-e0d4219e3a0c.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:68) ~[flink-rpc-akka_9ece02b9-c5ce-40c7-96a1-e0d4219e3a0c.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.lambda$withContextClassLoader$0(ClassLoadingUtils.java:41) ~[flink-rpc-akka_9ece02b9-c5ce-40c7-96a1-e0d4219e3a0c.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:49) [flink-rpc-akka_9ece02b9-c5ce-40c7-96a1-e0d4219e3a0c.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:48) [flink-rpc-akka_9ece02b9-c5ce-40c7-96a1-e0d4219e3a0c.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) [?:1.8.0_102]
   	at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) [?:1.8.0_102]
   	at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) [?:1.8.0_102]
   	at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157) [?:1.8.0_102]
   Caused by: org.apache.flink.client.deployment.application.ApplicationExecutionException: The application contains no execute() calls.
   	... 13 more
   2023-02-01 22:53:15,712 ERROR org.apache.flink.runtime.entrypoint.ClusterEntrypoint        [] - Fatal error occurred in the cluster entrypoint.
   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_102]
   	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308) ~[?:1.8.0_102]
   	at java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:943) ~[?:1.8.0_102]
   	at java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:926) ~[?:1.8.0_102]
   	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) ~[?:1.8.0_102]
   	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977) ~[?:1.8.0_102]
   	at org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap.runApplicationEntryPoint(ApplicationDispatcherBootstrap.java:299) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.client.deployment.application.ApplicationDispatcherBootstrap.lambda$runApplicationAsync$2(ApplicationDispatcherBootstrap.java:244) ~[flink-dist-1.15-vvr-6.0.4-SNAPSHOT.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_102]
   	at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[?:1.8.0_102]
   	at org.apache.flink.runtime.concurrent.akka.ActorSystemScheduledExecutorAdapter$ScheduledFutureTask.run(ActorSystemScheduledExecutorAdapter.java:171) ~[flink-rpc-akka_9ece02b9-c5ce-40c7-96a1-e0d4219e3a0c.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:68) ~[flink-rpc-akka_9ece02b9-c5ce-40c7-96a1-e0d4219e3a0c.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.lambda$withContextClassLoader$0(ClassLoadingUtils.java:41) ~[flink-rpc-akka_9ece02b9-c5ce-40c7-96a1-e0d4219e3a0c.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:49) [flink-rpc-akka_9ece02b9-c5ce-40c7-96a1-e0d4219e3a0c.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(ForkJoinExecutorConfigurator.scala:48) [flink-rpc-akka_9ece02b9-c5ce-40c7-96a1-e0d4219e3a0c.jar:1.15-vvr-6.0.4-SNAPSHOT]
   	at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) [?:1.8.0_102]
   	at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) [?:1.8.0_102]
   	at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) [?:1.8.0_102]
   	at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157) [?:1.8.0_102]
   Caused by: org.apache.flink.client.deployment.application.ApplicationExecutionException: The application contains no execute() calls.
   	... 13 more
   2023-02-01 22:53:15,717 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint        [] - Shutting KubernetesApplicationClusterEntrypoint down with application status UNKNOWN. Diagnostics Cluster entrypoint has been closed externally..
   ```
   
   


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

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


[GitHub] [hudi] Leoyzen commented on issue #7823: [SUPPORT]No execution calls after rollback compaction while using offline flink compactor.

Posted by "Leoyzen (via GitHub)" <gi...@apache.org>.
Leoyzen commented on issue #7823:
URL: https://github.com/apache/hudi/issues/7823#issuecomment-1413857325

   ```JAVA
         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");
             CompactionUtil.cleanInstant(table.getMetaClient(), instant);
             return;
           }
         }
   ```
   
   Why would it "return" instead of cleaning all the instants?


-- 
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 closed issue #7823: [SUPPORT]No execution calls after rollback compaction while using offline flink compactor.

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 closed issue #7823: [SUPPORT]No execution calls after rollback compaction while using offline flink compactor.
URL: https://github.com/apache/hudi/issues/7823


-- 
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 issue #7823: [SUPPORT]No execution calls after rollback compaction while using offline flink compactor.

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on issue #7823:
URL: https://github.com/apache/hudi/issues/7823#issuecomment-1415198302

   It's a legacy code, we can remove it, can you fire a PR 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