Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Hotfix][Zeta] Fix deploy operation timeout but task already finished bug #4867

Merged
merged 2 commits into from
Jun 5, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,13 @@
package org.apache.seatunnel.engine.server.checkpoint;

import org.apache.seatunnel.common.utils.ExceptionUtils;
import org.apache.seatunnel.common.utils.RetryUtils;
import org.apache.seatunnel.common.utils.SeaTunnelException;
import org.apache.seatunnel.engine.checkpoint.storage.PipelineState;
import org.apache.seatunnel.engine.checkpoint.storage.api.CheckpointStorage;
import org.apache.seatunnel.engine.common.Constant;
import org.apache.seatunnel.engine.common.config.server.CheckpointConfig;
import org.apache.seatunnel.engine.common.utils.ExceptionUtil;
import org.apache.seatunnel.engine.common.utils.PassiveCompletableFuture;
import org.apache.seatunnel.engine.core.checkpoint.Checkpoint;
import org.apache.seatunnel.engine.core.checkpoint.CheckpointIDCounter;
Expand All @@ -41,8 +44,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import com.hazelcast.map.IMap;
import com.hazelcast.spi.impl.operationservice.impl.InvocationFuture;
import lombok.Getter;
import lombok.NonNull;
import lombok.SneakyThrows;

import java.time.Instant;
Expand Down Expand Up @@ -131,6 +136,10 @@ public class CheckpointCoordinator {

private AtomicReference<String> errorByPhysicalVertex = new AtomicReference<>();

private final IMap<Object, Object> runningJobStateIMap;

private final String checkpointStateImapKey;

@SneakyThrows
public CheckpointCoordinator(
CheckpointManager manager,
Expand All @@ -140,13 +149,17 @@ public CheckpointCoordinator(
CheckpointPlan plan,
CheckpointIDCounter checkpointIdCounter,
PipelineState pipelineState,
ExecutorService executorService) {
ExecutorService executorService,
IMap<Object, Object> runningJobStateIMap,
boolean isStartWithSavePoint) {

this.executorService = executorService;
this.checkpointManager = manager;
this.checkpointStorage = checkpointStorage;
this.jobId = jobId;
this.pipelineId = plan.getPipelineId();
this.checkpointStateImapKey = "checkpoint_state_" + jobId + "_" + pipelineId;
this.runningJobStateIMap = runningJobStateIMap;
this.plan = plan;
this.coordinatorConfig = checkpointConfig;
this.tolerableFailureCheckpoints = coordinatorConfig.getTolerableFailureCheckpoints();
Expand Down Expand Up @@ -174,6 +187,27 @@ public CheckpointCoordinator(
serializer.deserialize(pipelineState.getStates(), CompletedCheckpoint.class);
}
this.checkpointCoordinatorFuture = new CompletableFuture();

// For job restore from master node active switch
CheckpointCoordinatorStatus checkpointCoordinatorStatus =
(CheckpointCoordinatorStatus) runningJobStateIMap.get(checkpointStateImapKey);

// This is not a new job
if (isStartWithSavePoint) {
updateStatus(CheckpointCoordinatorStatus.RUNNING);
return;
}

// If checkpointCoordinatorStatus is not null it means this CheckpointCoordinator is created
// by job restore from master node active switch
if (checkpointCoordinatorStatus != null) {
if (checkpointCoordinatorStatus.isEndState()) {
this.checkpointCoordinatorFuture.complete(
new CheckpointCoordinatorState(checkpointCoordinatorStatus, null));
} else {
updateStatus(CheckpointCoordinatorStatus.RUNNING);
}
}
}

public int getPipelineId() {
Expand Down Expand Up @@ -223,6 +257,7 @@ private void handleCoordinatorError(CheckpointCloseReason reason, Throwable e) {
return;
}
cleanPendingCheckpoint(reason);
updateStatus(CheckpointCoordinatorStatus.FAILED);
checkpointCoordinatorFuture.complete(
new CheckpointCoordinatorState(
CheckpointCoordinatorStatus.FAILED, errorByPhysicalVertex.get()));
Expand Down Expand Up @@ -307,6 +342,7 @@ protected void restoreCoordinator(boolean alreadyStarted) {
LOG.info("received restore CheckpointCoordinator with alreadyStarted= " + alreadyStarted);
errorByPhysicalVertex = new AtomicReference<>();
checkpointCoordinatorFuture = new CompletableFuture<>();
updateStatus(CheckpointCoordinatorStatus.RUNNING);
cleanPendingCheckpoint(CheckpointCloseReason.CHECKPOINT_COORDINATOR_RESET);
shutdown = false;
if (alreadyStarted) {
Expand Down Expand Up @@ -676,8 +712,15 @@ public synchronized void completePendingCheckpoint(CompletedCheckpoint completed
latestCompletedCheckpoint = completedCheckpoint;
if (isCompleted()) {
cleanPendingCheckpoint(CheckpointCloseReason.CHECKPOINT_COORDINATOR_COMPLETED);
checkpointCoordinatorFuture.complete(
new CheckpointCoordinatorState(CheckpointCoordinatorStatus.FINISHED, null));
if (latestCompletedCheckpoint.getCheckpointType().equals(SAVEPOINT_TYPE)) {
updateStatus(CheckpointCoordinatorStatus.SUSPEND);
checkpointCoordinatorFuture.complete(
new CheckpointCoordinatorState(CheckpointCoordinatorStatus.SUSPEND, null));
} else {
updateStatus(CheckpointCoordinatorStatus.FINISHED);
checkpointCoordinatorFuture.complete(
new CheckpointCoordinatorState(CheckpointCoordinatorStatus.FINISHED, null));
}
}
}

Expand Down Expand Up @@ -716,9 +759,36 @@ public PassiveCompletableFuture<CheckpointCoordinatorState> cancelCheckpoint() {
return new PassiveCompletableFuture<>(checkpointCoordinatorFuture);
}
cleanPendingCheckpoint(CheckpointCloseReason.PIPELINE_END);
updateStatus(CheckpointCoordinatorStatus.CANCELED);
CheckpointCoordinatorState checkpointCoordinatorState =
new CheckpointCoordinatorState(CheckpointCoordinatorStatus.CANCELED, null);
checkpointCoordinatorFuture.complete(checkpointCoordinatorState);
return new PassiveCompletableFuture<>(checkpointCoordinatorFuture);
}

private synchronized void updateStatus(@NonNull CheckpointCoordinatorStatus targetStatus) {
try {
RetryUtils.retryWithException(
() -> {
LOG.info(
String.format(
"Turn %s state from %s to %s",
checkpointStateImapKey,
runningJobStateIMap.get(checkpointStateImapKey),
targetStatus));
runningJobStateIMap.set(checkpointStateImapKey, targetStatus);
return null;
},
new RetryUtils.RetryMaterial(
Constant.OPERATION_RETRY_TIME,
true,
exception -> ExceptionUtil.isOperationNeedRetryException(exception),
Constant.OPERATION_RETRY_SLEEP));
} catch (Exception e) {
LOG.warn(
String.format(
"Set %s state %s to IMap failed, skip do it",
checkpointStateImapKey, targetStatus));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,18 @@
package org.apache.seatunnel.engine.server.checkpoint;

public enum CheckpointCoordinatorStatus {
RUNNING,

FINISHED,

CANCELED,

FAILED;
FAILED,

/** for savepoint job */
SUSPEND;

public boolean isEndState() {
return this == FINISHED || this == CANCELED || this == FAILED || this == SUSPEND;
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
return this == FINISHED || this == CANCELED || this == FAILED || this == SUSPEND;
return this != RUNNING;

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import org.apache.seatunnel.engine.server.task.statemachine.SeaTunnelTaskState;
import org.apache.seatunnel.engine.server.utils.NodeEngineUtil;

import com.hazelcast.map.IMap;
import com.hazelcast.spi.impl.NodeEngine;
import com.hazelcast.spi.impl.operationservice.impl.InvocationFuture;
import lombok.extern.slf4j.Slf4j;
Expand Down Expand Up @@ -84,7 +85,8 @@ public CheckpointManager(
JobMaster jobMaster,
Map<Integer, CheckpointPlan> checkpointPlanMap,
CheckpointConfig checkpointConfig,
ExecutorService executorService)
ExecutorService executorService,
IMap<Object, Object> runningJobStateIMap)
throws CheckpointStorageException {
this.executorService = executorService;
this.jobId = jobId;
Expand Down Expand Up @@ -130,7 +132,9 @@ public CheckpointManager(
plan,
idCounter,
pipelineState,
executorService);
executorService,
runningJobStateIMap,
isStartWithSavePoint);
} catch (Exception e) {
ExceptionUtil.sneakyThrow(e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -265,7 +265,7 @@ private TaskDeployState deployOnLocal(@NonNull SlotProfile slotProfile) throws E
});
}

private TaskDeployState deployOnRemote(@NonNull SlotProfile slotProfile) throws Exception {
private TaskDeployState deployOnRemote(@NonNull SlotProfile slotProfile) {
return deployInternal(
taskGroupImmutableInformation -> {
try {
Expand All @@ -281,7 +281,16 @@ private TaskDeployState deployOnRemote(@NonNull SlotProfile slotProfile) throws
slotProfile.getWorker())
.get();
} catch (Exception e) {
throw new RuntimeException(e);
if (getExecutionState().isEndState()) {
LOGGER.warning(ExceptionUtils.getMessage(e));
LOGGER.warning(
String.format(
"%s deploy error, but the state is already in end state %s, skip this error",
getTaskFullName(), currExecutionState));
return TaskDeployState.success();
} else {
return TaskDeployState.failed(e);
}
}
});
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -242,7 +242,8 @@ public void initCheckPointManager() throws CheckpointStorageException {
this,
checkpointPlanMap,
jobCheckpointConfig,
executorService);
executorService,
runningJobStateIMap);
}

// TODO replace it after ReadableConfig Support parse yaml format, then use only one config to
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -259,10 +259,6 @@ private CompletableFuture<Void> deployTask(PhysicalVertex task, SlotProfile slot
task.getTaskGroupLocation(),
ExecutionState.FAILED,
state.getThrowableMsg()));
throw new SeaTunnelEngineException(
String.format(
"deploy task %s failed, error msg: \n%s",
task.getTaskFullName(), state.getThrowableMsg()));
}
} catch (Exception e) {
throw new SeaTunnelEngineException(e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -341,7 +341,8 @@ public void ack(Barrier barrier) {
new TaskAcknowledgeOperation(
this.taskLocation,
(CheckpointBarrier) barrier,
checkpointStates.get(barrier.getId())));
checkpointStates.get(barrier.getId())))
.join();
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -224,7 +224,8 @@ public void triggerBarrier(Barrier barrier) throws Exception {
(CheckpointBarrier) barrier,
Collections.singletonList(
new ActionSubtaskState(
ActionStateKey.of(sink), -1, states))));
ActionStateKey.of(sink), -1, states))))
.join();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,8 @@ public void triggerBarrier(Barrier barrier) throws Exception {
new ActionSubtaskState(
ActionStateKey.of(source),
-1,
Collections.singletonList(serialize)))));
Collections.singletonList(serialize)))))
.join();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import java.util.concurrent.CompletableFuture;

import static org.apache.seatunnel.engine.common.Constant.IMAP_CHECKPOINT_ID;
import static org.apache.seatunnel.engine.common.Constant.IMAP_RUNNING_JOB_STATE;

@DisabledOnOs(OS.WINDOWS)
@Disabled
Expand Down Expand Up @@ -88,7 +89,8 @@ public void testHAByIMapCheckpointIDCounter() throws CheckpointStorageException
null,
planMap,
new CheckpointConfig(),
instance.getExecutorService("test"));
instance.getExecutorService("test"),
nodeEngine.getHazelcastInstance().getMap(IMAP_RUNNING_JOB_STATE));
Assertions.assertTrue(checkpointManager.isCompletedPipeline(1));
checkpointManager.listenPipeline(1, PipelineStatus.FINISHED);
Assertions.assertNull(checkpointIdMap.get(1));
Expand Down