This is an automated email from the ASF dual-hosted git repository.
lidongdai pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/seatunnel.git
The following commit(s) were added to refs/heads/dev by this push:
new 3b5483780d [Fix][Core] Cancel pending job (#8993)
3b5483780d is described below
commit 3b5483780dd0d9eee100b67f946debd19a9f6d6e
Author: Jast <[email protected]>
AuthorDate: Wed Mar 26 10:00:09 2025 +0800
[Fix][Core] Cancel pending job (#8993)
---
.../engine/server/CoordinatorService.java | 18 +-
.../engine/server/dag/physical/PhysicalPlan.java | 8 +-
...CoordinatorServiceWithCancelPendingJobTest.java | 185 +++++++++++++++++++++
.../src/test/resources/cancel_pending_job.conf | 70 ++++++++
4 files changed, 277 insertions(+), 4 deletions(-)
diff --git
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java
index 5a65f248ae..27129236ca 100644
---
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java
+++
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/CoordinatorService.java
@@ -156,7 +156,7 @@ public class CoordinatorService {
* key: job id; <br>
* value: job master;
*/
- private final Map<Long, Tuple2<PendingSourceState, JobMaster>>
pendingJobMasterMap =
+ protected final Map<Long, Tuple2<PendingSourceState, JobMaster>>
pendingJobMasterMap =
new ConcurrentHashMap<>();
/**
@@ -248,12 +248,19 @@ public class CoordinatorService {
Thread.sleep(3000);
return;
}
+
+ Long jobId = jobMaster.getJobId();
+
+ if (!pendingJobMasterMap.containsKey(jobId)) {
+ logger.fine(String.format("Job ID : %s already cancelled", jobId));
+ queueRemove(jobMaster);
+ return;
+ }
+
logger.fine(
String.format(
"Start pending job schedule, pendingJob Size : %s",
pendingJob.size()));
- Long jobId = jobMaster.getJobId();
-
logger.fine(
String.format(
"Start calculating whether pending task resources are
enough: %s", jobId));
@@ -744,6 +751,11 @@ public class CoordinatorService {
future.complete(null);
return new PassiveCompletableFuture<>(future);
} else {
+ // Cancel pending tasks
+ if (pendingJobMasterMap.containsKey(jobId)) {
+ pendingJobMasterMap.remove(jobId);
+ logger.fine(String.format("Cancel pending tasks : %s", jobId));
+ }
return new PassiveCompletableFuture<>(
CompletableFuture.supplyAsync(
() -> {
diff --git
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalPlan.java
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalPlan.java
index 9455d833e9..f5fd2fd4d7 100644
---
a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalPlan.java
+++
b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalPlan.java
@@ -199,7 +199,13 @@ public class PhysicalPlan {
return;
}
- updateJobState(JobStatus.CANCELING);
+ if (runningJobStateIMap.get(jobId) == JobStatus.PENDING) {
+ // The pending task needs to be directly set to 'cancelled' status
because it has not
+ // started running yet
+ updateJobState(JobStatus.CANCELED);
+ } else {
+ updateJobState(JobStatus.CANCELING);
+ }
}
public void savepointJob() {
diff --git
a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceWithCancelPendingJobTest.java
b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceWithCancelPendingJobTest.java
new file mode 100644
index 0000000000..a2b2d22340
--- /dev/null
+++
b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/CoordinatorServiceWithCancelPendingJobTest.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.seatunnel.engine.server;
+
+import org.apache.seatunnel.engine.common.config.EngineConfig;
+import org.apache.seatunnel.engine.common.config.SeaTunnelConfig;
+import org.apache.seatunnel.engine.common.config.server.ScheduleStrategy;
+import org.apache.seatunnel.engine.common.runtime.ExecutionMode;
+import org.apache.seatunnel.engine.common.utils.PassiveCompletableFuture;
+import org.apache.seatunnel.engine.core.dag.logical.LogicalDag;
+import org.apache.seatunnel.engine.core.job.JobImmutableInformation;
+import org.apache.seatunnel.engine.core.job.JobInfo;
+import org.apache.seatunnel.engine.core.job.JobStatus;
+import org.apache.seatunnel.engine.core.job.PipelineStatus;
+import org.apache.seatunnel.engine.server.dag.physical.PhysicalVertex;
+import org.apache.seatunnel.engine.server.dag.physical.PipelineLocation;
+import org.apache.seatunnel.engine.server.dag.physical.SubPlan;
+import org.apache.seatunnel.engine.server.execution.TaskGroupLocation;
+import org.apache.seatunnel.engine.server.master.JobMaster;
+import org.apache.seatunnel.engine.server.resourcemanager.resource.SlotProfile;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.condition.DisabledOnOs;
+import org.junit.jupiter.api.condition.OS;
+
+import com.hazelcast.config.Config;
+import com.hazelcast.internal.serialization.Data;
+import com.hazelcast.map.IMap;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.awaitility.Awaitility.await;
+
+/** JobMaster Tester. */
+@DisabledOnOs(OS.WINDOWS)
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+public class CoordinatorServiceWithCancelPendingJobTest extends
AbstractSeaTunnelServerTest {
+ /**
+ * IMap key is jobId and value is a Tuple2 Tuple2 key is JobMaster init
timestamp and value is
+ * the jobImmutableInformation which is sent by client when submit job
+ *
+ * <p>This IMap is used to recovery runningJobInfoIMap in JobMaster when a
new master node
+ * active
+ */
+ private IMap<Long, JobInfo> runningJobInfoIMap;
+
+ /**
+ * IMap key is one of jobId {@link PipelineLocation} and {@link
TaskGroupLocation}
+ *
+ * <p>The value of IMap is one of {@link JobStatus} {@link PipelineStatus}
{@link
+ * org.apache.seatunnel.engine.server.execution.ExecutionState}
+ *
+ * <p>This IMap is used to recovery runningJobStateIMap in JobMaster when
a new master node
+ * active
+ */
+ IMap<Object, Object> runningJobStateIMap;
+
+ /**
+ * IMap key is one of jobId {@link PipelineLocation} and {@link
TaskGroupLocation}
+ *
+ * <p>The value of IMap is one of {@link
+ * org.apache.seatunnel.engine.server.dag.physical.PhysicalPlan}
stateTimestamps {@link SubPlan}
+ * stateTimestamps {@link PhysicalVertex} stateTimestamps
+ *
+ * <p>This IMap is used to recovery runningJobStateTimestampsIMap in
JobMaster when a new master
+ * node active
+ */
+ IMap<Object, Long[]> runningJobStateTimestampsIMap;
+
+ /**
+ * IMap key is {@link PipelineLocation}
+ *
+ * <p>The value of IMap is map of {@link TaskGroupLocation} and the {@link
SlotProfile} it used.
+ *
+ * <p>This IMap is used to recovery ownedSlotProfilesIMap in JobMaster
when a new master node
+ * active
+ */
+ private IMap<PipelineLocation, Map<TaskGroupLocation, SlotProfile>>
ownedSlotProfilesIMap;
+
+ @BeforeAll
+ public void before() {
+ String name = this.getClass().getName();
+ Config hazelcastConfig = Config.loadFromString(getHazelcastConfig());
+ hazelcastConfig.setClusterName(
+ TestUtils.getClusterName("AbstractSeaTunnelServerTest_" +
name));
+ SeaTunnelConfig seaTunnelConfig = loadSeaTunnelConfig();
+ seaTunnelConfig.setHazelcastConfig(hazelcastConfig);
+ EngineConfig engineConfig = seaTunnelConfig.getEngineConfig();
+ engineConfig.setMode(ExecutionMode.LOCAL);
+ engineConfig.setScheduleStrategy(ScheduleStrategy.WAIT);
+ engineConfig.getSlotServiceConfig().setDynamicSlot(false);
+ engineConfig.getSlotServiceConfig().setSlotNum(1);
+ instance =
SeaTunnelServerStarter.createHazelcastInstance(seaTunnelConfig);
+ nodeEngine = instance.node.nodeEngine;
+ server = nodeEngine.getService(SeaTunnelServer.SERVICE_NAME);
+ LOGGER = nodeEngine.getLogger(AbstractSeaTunnelServerTest.class);
+ }
+
+ @Test
+ public void testCancelPendingJob() throws InterruptedException {
+
+ long jobId =
instance.getFlakeIdGenerator("testCancelPendingJob").newId();
+ JobMaster jobMaster = newJobInstanceWithRunningState(jobId);
+
+ // Verify that the task is pending
+ Assertions.assertTrue(
+
server.getCoordinatorService().pendingJobMasterMap.containsKey(jobId));
+
+ // Cancel Task
+ PassiveCompletableFuture<Void> voidPassiveCompletableFuture =
+ server.getCoordinatorService().cancelJob(jobId);
+ voidPassiveCompletableFuture.join();
+
+ // Verify if the task has been deleted in pending
+ Assertions.assertFalse(
+
server.getCoordinatorService().pendingJobMasterMap.containsKey(jobId));
+
+ // Verify if the final status of the task is cancelled
+ await().atMost(120, TimeUnit.SECONDS)
+ .untilAsserted(
+ () ->
+ Assertions.assertEquals(
+ JobStatus.CANCELED,
jobMaster.getJobStatus()));
+ }
+
+ private JobMaster newJobInstanceWithRunningState(long jobId) throws
InterruptedException {
+ return newJobInstanceWithRunningState(jobId, false);
+ }
+
+ private JobMaster newJobInstanceWithRunningState(long jobId, boolean
restore)
+ throws InterruptedException {
+ LogicalDag testLogicalDag =
+ TestUtils.createTestLogicalPlan(
+ "cancel_pending_job.conf", "cancel_pending_job",
jobId);
+
+ JobImmutableInformation jobImmutableInformation =
+ new JobImmutableInformation(
+ jobId,
+ "Test",
+ restore,
+ nodeEngine.getSerializationService(),
+ testLogicalDag,
+ Collections.emptyList(),
+ Collections.emptyList());
+
+ Data data =
nodeEngine.getSerializationService().toData(jobImmutableInformation);
+
+ PassiveCompletableFuture<Void> voidPassiveCompletableFuture =
+ server.getCoordinatorService()
+ .submitJob(jobId, data,
jobImmutableInformation.isStartWithSavePoint());
+ voidPassiveCompletableFuture.join();
+
+ JobMaster jobMaster =
server.getCoordinatorService().getJobMaster(jobId);
+
+ // waiting for job status turn to running
+ await().atMost(120, TimeUnit.SECONDS)
+ .untilAsserted(
+ () -> Assertions.assertEquals(JobStatus.PENDING,
jobMaster.getJobStatus()));
+
+ // Because handleCheckpointTimeout is an async method, so we need
sleep 5s to waiting job
+ // status become running again
+ Thread.sleep(5000);
+ return jobMaster;
+ }
+}
diff --git
a/seatunnel-engine/seatunnel-engine-server/src/test/resources/cancel_pending_job.conf
b/seatunnel-engine/seatunnel-engine-server/src/test/resources/cancel_pending_job.conf
new file mode 100644
index 0000000000..aa620baccf
--- /dev/null
+++
b/seatunnel-engine/seatunnel-engine-server/src/test/resources/cancel_pending_job.conf
@@ -0,0 +1,70 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+######
+###### This config file is a demonstration of streaming processing in
seatunnel config
+######
+
+env {
+ parallelism = 1
+ job.mode = "BATCH"
+}
+
+source {
+ # This is a example source plugin **only for test and demonstrate the
feature source plugin**
+ FakeSource {
+ plugin_output = "fake"
+ schema = {
+ fields {
+ name = "string"
+ age = "int"
+ }
+ }
+ parallelism = 1
+ }
+
+ FakeSource {
+ plugin_output = "fake2"
+ schema = {
+ fields {
+ name = "string"
+ age = "int"
+ }
+ }
+ parallelism = 1
+ }
+}
+
+transform {
+}
+
+sink {
+ LocalFile {
+ path="/tmp/hive/warehouse/test2"
+ field_delimiter="\t"
+ row_delimiter="\n"
+ partition_by=["age"]
+ partition_dir_expression="${k0}=${v0}"
+ is_partition_field_write_in_file=true
+ file_name_expression="${transactionId}_${now}"
+ file_format_type="text"
+ sink_columns=["name","age"]
+ filename_time_format="yyyy.MM.dd"
+ is_enable_transaction=true
+ save_mode="error",
+ plugin_input="fake,fake2"
+ }
+}
\ No newline at end of file