Skip to content

[Fix][Zeta] Make deployTask idempotent during master failover recovery#10567

Merged
xiaochen-zhou merged 6 commits intoapache:devfrom
ricky2129:fix/zeta-task-group-idempotent-deploy-master-failover
Mar 15, 2026
Merged

[Fix][Zeta] Make deployTask idempotent during master failover recovery#10567
xiaochen-zhou merged 6 commits intoapache:devfrom
ricky2129:fix/zeta-task-group-idempotent-deploy-master-failover

Conversation

@ricky2129
Copy link
Contributor

During master rolling restart, workers keep tasks running in executionContexts. The new master restores state and tries to re-deploy tasks that never stopped, causing TaskGroupLocation already exists error, deploy failure, infinite restore loop, and jobs showing FAILED in UI while still writing to sink.

Fix: when deployTask finds a TaskGroupLocation already in executionContexts (task actively running), return success instead of throwing. The worker will notify the master of the terminal state via NotifyTaskStatusOperation when the task eventually completes, maintaining correct lifecycle tracking.

Legitimate redeploy is unaffected: completed tasks are moved to finishedExecutionContexts by taskDone(), so executionContexts.containsKey() returns false and normal deploy proceeds.

Purpose of this pull request

Does this PR introduce any user-facing change?

How was this patch tested?

Check list

@DanielCarter-stack
Copy link

Issue 1: TOCTOU Race Condition (Theoretical Risk)

Location: TaskExecutionService.java:333-349

synchronized (this) {
    if (executionContexts.containsKey(taskGroup.getTaskGroupLocation())) {
        logger.warning(...);
        return TaskDeployState.success();
    }
    deployLocalTask(taskGroup, classLoaders, taskJars);
    return TaskDeployState.success();
}

Related Context:

  • Caller: PhysicalVertex.deployInternal() :320-327
  • Caller: DeployTaskOperation.runInternal() :49-56
  • Collaborating class: TaskGroupExecutionTracker.taskDone() :936-986

Problem Description:

There is theoretically a TOCTOU (Time-of-Check-Time-of-Use) race condition:

线程 A (Master failover):          线程 B (Task completion):
--------------------------------    --------------------------------
1. 检查 executionContexts.containsKey() → false
                                   2. taskDone() 被调用
                                      executionContexts.remove(location)
                                      finishedExecutionContexts.put(location)
3. deployLocalTask() → 插入新的 executionContext

Potential Risks:

  • Risk 1: If the task happens to complete between the check and deployment, a new executionContext will be created, but the old TaskGroupContext might still be in finishedExecutionContexts
  • Risk 2: May lead to resource leaks (ClassLoader, cancellationFutures, etc.)

Impact Scope:

  • Direct Impact: TaskExecutionService.deployTask(), deployLocalTask()
  • Indirect Impact: executionContexts, finishedExecutionContexts, cancellationFutures, taskAsyncFunctionFuture
  • Affected Area: Core framework, all jobs running on Zeta Engine

Severity: MINOR

Rationale:

  • ✅ Within a single JVM, synchronized (this) already protects the entire critical section
  • ✅ Thread B's taskDone() is also in a synchronized block (implicit, through serial execution of CompletableFuture)
  • ✅ The actual window for triggering this race is extremely small (microsecond level)
  • ✅ Even if triggered, the impact is minimal (a new Context is created to replace the old one)
  • ⚠️ However, this behavior is not documented or commented

Improvement Suggestions:

No code changes needed, but comments are recommended:

synchronized (this) {
    // Note: If the task completes between the containsKey check and deployLocalTask,
    // a new executionContext will be created. This is acceptable because:
    // 1. The old context is moved to finishedExecutionContexts atomically by taskDone()
    // 2. Resources are cleaned up correctly (cancellationFutures, ClassLoaders)
    // 3. The new deployment proceeds normally
    // 4. This window is extremely small due to synchronized protection
    if (executionContexts.containsKey(taskGroup.getTaskGroupLocation())) {
        logger.warning(
            String.format(
                "TaskGroupLocation %s already exists and is active, "
                    + "skipping redeploy for master failover recovery",
                taskGroup.getTaskGroupLocation()));
        return TaskDeployState.success();
    }
    deployLocalTask(taskGroup, classLoaders, taskJars);
    return TaskDeployState.success();
}

Issue 2: Risk of Silent Failure in Error Scenarios

Location: TaskExecutionService.java:334-346

if (executionContexts.containsKey(taskGroup.getTaskGroupLocation())) {
    logger.warning(...);
    return TaskDeployState.success();
}

Related Context:

  • Status check: PhysicalVertex.initStateFuture() :182-205
  • Status validation: PhysicalVertex.checkTaskGroupIsExecuting() :212-249
  • Exception handling: TaskGroupExecutionTracker.taskDone() :936-986

Problem Description:

If executionContexts.containsKey() returns true, but the task is actually in a "zombie" state (process alive but task thread stuck or deadlocked):

  • Returns TaskDeployState.success()
  • Master believes the task is running and maintains RUNNING status
  • But the task is actually unable to complete and will never send NotifyTaskStatusOperation
  • Causes the job to hang permanently

Potential Risks:

  • Risk 1: Job displays as RUNNING in UI but is actually stuck
  • Risk 2: User waits for job completion that will never happen
  • Risk 3: Data pipeline interrupted but not detected

Impact Scope:

  • Direct Impact: All jobs experiencing thread stuck, deadlock, or infinite loop
  • Indirect Impact: Monitoring and alerting system (may not detect this silent failure)
  • Affected Area: Core framework, all SeaTunnel jobs

Severity: MINOR

Rationale:

  • ✅ This issue existed before the modification (old logic threw exceptions, but master would retry, potentially timing out eventually)
  • ✅ SeaTunnel has other mechanisms to detect task health:
    • CheckTaskGroupIsExecutingOperation validates whether the task is executing
    • Heartbeat mechanism (if configured)
    • Users can manually cancel stuck tasks
  • ✅ This "zombie task" scenario is rare in the first place (usually caused by code bugs or resource issues)
  • ⚠️ However, after the fix, this issue becomes more hidden (no more exception logs)

Improvement Suggestions:

  1. Add task health check:
if (executionContexts.containsKey(taskGroup.getTaskGroupLocation())) {
    TaskGroupContext existingContext = executionContexts.get(taskGroup.getTaskGroupLocation());
    
    // Optional: Verify task is still healthy by checking its last activity time
    // This requires adding a lastActivityTime field to TaskGroupContext
    
    logger.warning(
        String.format(
            "TaskGroupLocation %s already exists and is active, "
                + "skipping redeploy for master failover recovery",
            taskGroup.getTaskGroupLocation()));
    return TaskDeployState.success();
}
  1. Add timeout detection on master side:

PhysicalVertex.initStateFuture() already has checks (line 195-202):

if (ExecutionState.RUNNING.equals(currExecutionState)) {
    if (!checkTaskGroupIsExecuting(taskGroupLocation)) {
        updateTaskState(ExecutionState.FAILING);
    }
}

Additional timeout checks are recommended:

// After some timeout, if still RUNNING but no progress, mark as failed
  1. Document this scenario:

Explain in JavaDoc:

/**
 * ...
 * <p><b>Note:</b> If a task is in executionContexts but has become unresponsive
 * (zombie state), this method will still return success. The master relies on
 * other mechanisms (heartbeats, timeouts, health checks) to detect and handle
 * such scenarios.
 */

Issue 3: Missing Test Cases

Location: TaskExecutionServiceTest.java

Related Context:

  • Existing tests: TaskExecutionServiceTest.java
  • Test framework: JUnit 5, Awaitility
  • Mock support: AbstractSeaTunnelServerTest

Problem Description:

The current test suite does not cover idempotent deployment behavior under master failover scenarios. Missing test scenarios:

  1. Scenario 1: Master failover - Redeploy running task

    • Deploy task for the first time
    • Simulate master failover, deploy the same TaskGroupLocation again
    • Verify return success (no exception thrown)
    • Verify task is still in executionContexts
  2. Scenario 2: Redeploy completed task

    • Deploy task and wait for completion
    • Task moves to finishedExecutionContexts
    • Deploy the same TaskGroupLocation again
    • Verify return success (new deployment)
    • Verify new executionContexts is created
  3. Scenario 3: Concurrent deployment (multi-threaded)

    • Multiple threads call deployTask() for the same TaskGroupLocation simultaneously
    • Verify only one is actually deployed
    • Verify other threads receive success return

Potential Risks:

  • Risk 1: Future modifications may break this idempotency
  • Risk 2: Code refactoring may introduce regressions
  • Risk 3: Edge cases remain undiscovered

Impact Scope:

  • Direct Impact: Test coverage and code quality assurance
  • Indirect Impact: Confidence in future maintenance and refactoring
  • Affected Area: Single file (test file)

Severity: MAJOR

Rationale:

  • ❌ This is a core functionality modification (idempotency)
  • ❌ Lack of tests reduces code maintainability
  • ❌ "how was this patch tested?" section in PR description is empty
  • ⚠️ Although existing tests may indirectly cover some scenarios, they are not explicit and complete
  • ✅ But this does not affect the correctness of the current fix (code logic is correct)

Improvement Suggestions:

Add the following test cases to TaskExecutionServiceTest.java:

@Test
public void testRedeployDuringMasterFailover() throws InterruptedException {
    TaskExecutionService taskExecutionService = server.getTaskExecutionService();
    
    AtomicBoolean stop = new AtomicBoolean(false);
    long sleepTime = 5000; // Long enough to simulate a running task
    
    TestTask testTask = new TestTask(stop, sleepTime, true);
    TaskGroupLocation location = new TaskGroupLocation(jobId, pipeLineId, FLAKE_ID_GENERATOR.newId());
    TaskGroupDefaultImpl taskGroup = new TaskGroupDefaultImpl(
        location, "ts", Lists.newArrayList(testTask));
    
    // Create TaskGroupImmutableInformation
    List<Data> tasksData = Arrays.asList(
        nodeEngine.getSerializationService().toData(testTask));
    TaskGroupImmutableInformation taskInfo = new TaskGroupImmutableInformation(
        jobId,
        1,
        TaskGroupType.INTERMEDIATE_BLOCKING_QUEUE,
        location,
        "testRedeployDuringMasterFailover",
        tasksData,
        Arrays.asList(Collections.emptySet()),
        Arrays.asList(emptySet()));
    
    Data data = nodeEngine.getSerializationService().toData(taskInfo);
    
    // 1. First deployment
    TaskDeployState state1 = taskExecutionService.deployTask(data);
    assertTrue(state1.isSuccess());
    assertNotNull(taskExecutionService.getActiveExecutionContext(location));
    
    // 2. Simulate master failover: redeploy the same task
    TaskDeployState state2 = taskExecutionService.deployTask(data);
    assertTrue(state2.isSuccess(), "Second deployment should return success");
    
    // 3. Verify task is still in executionContexts (not moved to finished)
    assertNotNull(taskExecutionService.getActiveExecutionContext(location));
    
    // 4. Let the task complete
    stop.set(true);
    await().atMost(sleepTime + 5000, TimeUnit.MILLISECONDS)
        .untilAsserted(() -> {
            TaskGroupContext ctx = taskExecutionService.getExecutionContext(location);
            // Task should be in finishedExecutionContexts after completion
            assertNotNull(ctx);
        });
}

@Test
public void testRedeployAfterTaskCompletion() {
    TaskExecutionService taskExecutionService = server.getTaskExecutionService();
    
    AtomicBoolean stop = new AtomicBoolean(false);
    long sleepTime = 100;
    
    TestTask testTask = new TestTask(stop, sleepTime, true);
    TaskGroupLocation location = new TaskGroupLocation(jobId, pipeLineId, FLAKE_ID_GENERATOR.newId());
    
    // Deploy and wait for completion
    CompletableFuture<TaskExecutionState> future = deployLocalTask(
        taskExecutionService,
        new TaskGroupDefaultImpl(location, "ts", Lists.newArrayList(testTask)));
    
    stop.set(true);
    await().atMost(sleepTime + 5000, TimeUnit.MILLISECONDS)
        .untilAsserted(() -> {
            assertEquals(FINISHED, future.get().getExecutionState());
        });
    
    // Verify task is in finishedExecutionContexts
    assertNotNull(taskExecutionService.getExecutionContext(location));
    try {
        taskExecutionService.getActiveExecutionContext(location);
        fail("Task should not be in executionContexts after completion");
    } catch (TaskGroupContextNotFoundException e) {
        // Expected
    }
    
    // Create new TaskGroupImmutableInformation for redeployment
    List<Data> tasksData = Arrays.asList(
        nodeEngine.getSerializationService().toData(testTask));
    TaskGroupImmutableInformation taskInfo = new TaskGroupImmutableInformation(
        jobId,
        1,
        TaskGroupType.INTERMEDIATE_BLOCKING_QUEUE,
        location,
        "testRedeployAfterTaskCompletion",
        tasksData,
        Arrays.asList(Collections.emptySet()),
        Arrays.asList(emptySet()));
    
    Data data = nodeEngine.getSerializationService().toData(taskInfo);
    
    // Redeploy the completed task
    TaskDeployState state = taskExecutionService.deployTask(data);
    assertTrue(state.isSuccess(), "Redeploy of completed task should succeed");
    
    // Verify new executionContext is created
    assertNotNull(taskExecutionService.getActiveExecutionContext(location));
    
    // Cleanup
    taskExecutionService.cancelTaskGroup(location);
}

Issue 4: Missing JavaDoc Documentation

Location: TaskExecutionService.java:276

public TaskDeployState deployTask(@NonNull TaskGroupImmutableInformation taskImmutableInfo)

Related Context:

  • Caller: DeployTaskOperation.runInternal() :49-56
  • Collaborating class: NotifyTaskStatusOperation :31-80
  • Related method: getActiveExecutionContext() :200-208

Problem Description:

The deployTask() method lacks JavaDoc and does not explain:

  1. The method's idempotency特性
  2. Behavior under master failover scenarios
  3. Conditions for returning success (first deployment successful OR task already running)
  4. Relationship with NotifyTaskStatusOperation

Potential Risks:

  • Risk 1: Other developers may mistake this for a "duplicate deployment error" and "fix" it
  • Risk 2: Callers are unclear about idempotency guarantees and may add unnecessary checks
  • Risk 3: Increased maintenance cost (need to understand behavior through code)

Impact Scope:

  • Direct Impact: Code maintainability and comprehensibility
  • Indirect Impact: Future refactoring and extension
  • Affected Area: Documentation of a single method

Severity: MINOR

Rationale:

  • ⚠️ Although the code logic is correct, lack of documentation reduces maintainability
  • ✅ Comments within the method (line 335-340) explain the behavior
  • ✅ PR description clearly explains the problem
  • ❌ But JavaDoc is part of the API contract and should be complete

Improvement Suggestions:

Add complete JavaDoc:

/**
 * Deploys a task group to this worker node. 
 *
 * <p><b>Idempotency:</b> This operation is idempotent with respect to the task group location.
 * If a task group with the same {@link TaskGroupLocation} is already actively running in
 * {@code executionContexts}, this method returns {@link TaskDeployState#success()} without
 * redeploying. This behavior supports master failover recovery where the new master may
 * attempt to redeploy tasks that never stopped on the worker.
 *
 * <p><b>Master Failover Scenario:</b> During master rolling restart or failover:
 * <ul>
 *   <li>Workers keep tasks running in {@code executionContexts}</li>
 *   <li>The new master restores state and attempts to redeploy tasks</li>
 *   <li>This method detects the duplicate and returns success</li>
 *   <li>The master reconnects without interrupting the running task</li>
 *   <li>When the task completes, the worker notifies the master via
 *       {@link NotifyTaskStatusOperation}</li>
 * </ul>
 *
 * <p><b>Legitimate Redeployment:</b> Completed tasks are moved to {@code finishedExecutionContexts}
 * by {@link TaskGroupExecutionTracker#taskDone(Task)}, so {@code executionContexts.containsKey()}
 * returns false and normal deploy proceeds.
 *
 * <p><b>Thread Safety:</b> This method is synchronized on {@code this} to ensure atomic
 * check-and-deploy semantics.
 *
 * @param taskImmutableInformation the immutable information of task group to deploy
 * @return {@link TaskDeployState#success()} if deployment succeeded or task already running;
 *         {@link TaskDeployState#failed(Throwable)} if deployment failed
 * @throws RuntimeException if deployment fails (caught and converted to failed state)
 */
public TaskDeployState deployTask(@NonNull TaskGroupImmutableInformation taskImmutableInfo)

Issue 5: Incomplete Log Tracing Information

Location: TaskExecutionService.java:341-345

logger.warning(
    String.format(
        "TaskGroupLocation %s already exists and is active, "
            + "skipping redeploy for master failover recovery",
        taskGroup.getTaskGroupLocation()));

Related Context:

  • Logging system: Hazelcast ILogger
  • Tracing mechanism: TracingOperation (parent class)
  • Related logs: notifyTaskStatusToMaster() :450-490

Problem Description:

Current logs lack the following tracing information:

  1. ExecutionId: taskImmutableInfo.getExecutionId() can help correlate different deployment attempts
  2. JobId: Although TaskGroupLocation contains JobId, it is not explicitly logged
  3. Worker node information: No record of which worker's log this is

Potential Risks:

  • Risk 1: Difficult to trace the same deployment request across nodes in distributed environments
  • Risk 2: Manual correlation from multiple logs required when troubleshooting
  • Risk 3: Unable to quickly count master failover events

Impact Scope:

  • Direct Impact: Observability and troubleshooting efficiency
  • Indirect Impact: Operations and monitoring
  • Affected Area: Logging system

Severity: MINOR

Rationale:

  • ✅ TaskGroupLocation itself already contains a lot of information
  • ✅ Hazelcast's log context may contain node information
  • ⚠️ But adding more explicit information can improve observability
  • ⚠️ In production environments, additional tracing fields are important

Improvement Suggestions:

Enhance log content:

logger.warning(
    String.format(
        "TaskGroupLocation %s already exists and is active, "
            + "skipping redeploy for master failover recovery. "
            + "ExecutionId: %s, JobId: %d, PipelineId: %d",
        taskGroup.getTaskGroupLocation(),
        taskImmutableInfo.getExecutionId(),
        taskGroup.getTaskGroupLocation().getJobId(),
        taskGroup.getTaskGroupLocation().getPipelineId()));

Or use structured logging (if supported):

logger.warning(
    String.format(
        "TaskGroupLocation %s already exists and is active, skipping redeploy. " +
        "reason=master_failover_recovery, executionId=%s, jobId=%d, pipelineId=%d",
        taskGroup.getTaskGroupLocation(),
        taskImmutableInfo.getExecutionId(),
        taskGroup.getTaskGroupLocation().getJobId(),
        taskGroup.getTaskGroupLocation().getPipelineId()));

@ricky2129
Copy link
Contributor Author

@dybyte can you help with this??

Copy link
Collaborator

@dybyte dybyte left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please add test cases for this change?

@ricky2129
Copy link
Contributor Author

@dybyte added the test cases for the change. Please review and give suggestions

@ricky2129 ricky2129 force-pushed the fix/zeta-task-group-idempotent-deploy-master-failover branch from 6cea96c to 2b908cc Compare March 10, 2026 15:43
Copy link
Collaborator

@dybyte dybyte left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM if CI passes. Thanks @ricky2129 !

@ricky2129 ricky2129 force-pushed the fix/zeta-task-group-idempotent-deploy-master-failover branch from 920cf70 to 7813f79 Compare March 11, 2026 13:36
During master rolling restart, workers keep tasks running in executionContexts.
The new master restores state and tries to re-deploy tasks that never stopped,
causing TaskGroupLocation already exists error, deploy failure, infinite restore
loop, and jobs showing FAILED in UI while still writing to sink.

Fix: when deployTask finds a TaskGroupLocation already in executionContexts
(task actively running), return success instead of throwing. The worker will
notify the master of the terminal state via NotifyTaskStatusOperation when the
task eventually completes, maintaining correct lifecycle tracking.

Legitimate redeploy is unaffected: completed tasks are moved to
finishedExecutionContexts by taskDone(), so executionContexts.containsKey()
returns false and normal deploy proceeds.
…ocess

During master failover, IMap may still hold DEPLOYING state if the master
crashed after the worker finished deploying but before the master processed
NotifyTaskStatusOperation and updated IMap to RUNNING.

restoreExecutionState() previously called stateProcess() with whatever
currExecutionState was set at construction time, causing stateProcess() to
enter the DEPLOYING branch and send a redundant DeployTaskOperation to the
worker — which already has the task running in executionContexts.

Fix: re-read currExecutionState from IMap in restoreExecutionState(), then
check whether the task is actually executing on the worker. If DEPLOYING but
the task is confirmed running, advance state to RUNNING before calling
stateProcess() so no redundant deploy is sent.

This is the master-side complement to the worker-side idempotent deployTask()
fix. Together they make master failover recovery fully robust:
- Master side: avoids sending the redundant DeployTaskOperation
- Worker side: handles it gracefully if one is sent anyway
…nState fix

- testDeployTaskIdempotentWhenAlreadyRunning: verifies that calling
  deployTask() twice for the same active TaskGroupLocation returns
  TaskDeployState.success() on both calls and does not interrupt the
  running task (before the fix the second call threw RuntimeException)

- testRestoreExecutionStateAdvancesDeployingToRunningWhenTaskIsExecuting:
  verifies that restoreExecutionState() advances IMap state from DEPLOYING
  to RUNNING when CheckTaskGroupIsExecutingOperation confirms the task is
  already executing on the worker, so stateProcess() does not send a
  redundant DeployTaskOperation
…-side idempotency only

Worker-side deployTask() idempotency is sufficient for master failover recovery.
The master-side checkTaskGroupIsExecuting() pre-check required an extra network
round-trip without meaningful benefit, so it is removed. The test for it is also
removed; testDeployTaskIdempotentWhenAlreadyRunning covers the remaining fix.
@ricky2129 ricky2129 force-pushed the fix/zeta-task-group-idempotent-deploy-master-failover branch from 7813f79 to a6ca331 Compare March 11, 2026 13:45
ricky2129 and others added 2 commits March 13, 2026 23:29
Co-authored-by: David Zollo <davidzollo365@gmail.com>
…Service

GitHub UI applied reviewer suggestion with 4-space continuation indent
instead of the required 8-space continuation indent for method call args.
Spotless requires 8-space (double-indent) for wrapped method parameters.
@ricky2129
Copy link
Contributor Author

@davidzollo added the changes.

Copy link
Contributor

@davidzollo davidzollo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1
Good job

Copy link
Contributor

@xiaochen-zhou xiaochen-zhou left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

@xiaochen-zhou xiaochen-zhou merged commit e365ebf into apache:dev Mar 15, 2026
4 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants