diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java
index d5ca307a00cd6..dee3edfc578c7 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java
@@ -439,6 +439,11 @@ private void handleEndInputEvent(WriteMetadataEvent event) {
Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
// sync Hive synchronously if it is enabled in batch mode.
syncHive();
+ // schedules the compaction plan in batch execution mode
+ if (tableState.scheduleCompaction) {
+ // if async compaction is on, schedule the compaction
+ CompactionUtil.scheduleCompaction(metaClient, writeClient, tableState.isDeltaTimeCompaction, true);
+ }
}
}
}
diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java
index d5e718883b86c..efaf4bfdc75b0 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java
@@ -18,6 +18,7 @@
package org.apache.hudi.sink.compact;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -47,7 +48,7 @@
*
It should be singleton to avoid conflicts.
*/
public class CompactionPlanOperator extends AbstractStreamOperator
- implements OneInputStreamOperator