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][ST-Engine] fix job restart of all node down #3722

Merged
merged 1 commit into from
Dec 15, 2022
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 @@ -249,33 +249,38 @@ public PassiveCompletableFuture<TaskExecutionState> deployLocalTask(
resultFuture.whenComplete(withTryCatch(logger, (r, s) -> {
logger.info(
String.format("Task %s complete with state %s", r.getTaskGroupLocation(), r.getExecutionState()));
long sleepTime = 1000;
boolean notifyStateSuccess = false;
while (isRunning && !notifyStateSuccess) {
InvocationFuture<Object> invoke = nodeEngine.getOperationService().createInvocationBuilder(
SeaTunnelServer.SERVICE_NAME,
new NotifyTaskStatusOperation(taskGroup.getTaskGroupLocation(), r),
nodeEngine.getMasterAddress()).invoke();
notifyTaskStatusToMaster(taskGroup.getTaskGroupLocation(), r);
}));
return new PassiveCompletableFuture<>(resultFuture);
}

@SuppressWarnings("checkstyle:MagicNumber")
private void notifyTaskStatusToMaster(TaskGroupLocation taskGroupLocation, TaskExecutionState taskExecutionState){
long sleepTime = 1000;
boolean notifyStateSuccess = false;
while (isRunning && !notifyStateSuccess) {
InvocationFuture<Object> invoke = nodeEngine.getOperationService().createInvocationBuilder(
SeaTunnelServer.SERVICE_NAME,
new NotifyTaskStatusOperation(taskGroupLocation, taskExecutionState),
nodeEngine.getMasterAddress()).invoke();
try {
invoke.get();
notifyStateSuccess = true;
} catch (InterruptedException e) {
logger.severe(e);
Thread.interrupted();
} catch (ExecutionException e) {
logger.warning(ExceptionUtils.getMessage(e));
logger.warning(String.format("notify the job of the task(%s) status failed, retry in %s millis",
taskGroupLocation, sleepTime));
try {
invoke.get();
notifyStateSuccess = true;
} catch (InterruptedException e) {
Thread.sleep(sleepTime);
} catch (InterruptedException ex) {
logger.severe(e);
Thread.interrupted();
} catch (ExecutionException e) {
logger.warning(ExceptionUtils.getMessage(e));
logger.warning(String.format("notify the job of the task(%s) status failed, retry in %s millis",
taskGroup.getTaskGroupLocation(), sleepTime));
try {
Thread.sleep(sleepTime);
} catch (InterruptedException ex) {
logger.severe(e);
Thread.interrupted();
}
}
}
}));
return new PassiveCompletableFuture<>(resultFuture);
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.seatunnel.engine.server.resourcemanager.resource.SlotProfile;
import org.apache.seatunnel.engine.server.task.TaskGroupImmutableInformation;
import org.apache.seatunnel.engine.server.task.operation.CancelTaskOperation;
import org.apache.seatunnel.engine.server.task.operation.CheckTaskGroupIsExecutingOperation;
import org.apache.seatunnel.engine.server.task.operation.DeployTaskOperation;

import com.hazelcast.cluster.Address;
Expand All @@ -39,11 +40,14 @@
import com.hazelcast.logging.Logger;
import com.hazelcast.map.IMap;
import com.hazelcast.spi.impl.NodeEngine;
import com.hazelcast.spi.impl.operationservice.impl.InvocationFuture;
import lombok.NonNull;

import java.net.URL;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.function.Consumer;

Expand Down Expand Up @@ -177,22 +181,59 @@ public PhysicalVertex(int subTaskGroupIndex,
this.runningJobStateTimestampsIMap = runningJobStateTimestampsIMap;
}

@SuppressWarnings("checkstyle:MagicNumber")
public PassiveCompletableFuture<TaskExecutionState> initStateFuture() {
this.taskFuture = new CompletableFuture<>();
ExecutionState executionState = (ExecutionState) runningJobStateIMap.get(taskGroupLocation);
if (executionState != null) {
LOGGER.info(
String.format("The task %s is in state %s when init state future", taskFullName, executionState));
}
// if the task state is RUNNING
// We need to check the real running status of Task from taskExecutionServer.
// Because the state may be RUNNING when the cluster is restarted, but the Task no longer exists.
if (ExecutionState.RUNNING.equals(executionState)){
if (!checkTaskGroupIsExecuting(taskGroupLocation)) {
this.taskFuture.complete(new TaskExecutionState(taskGroupLocation, ExecutionState.FAILED, null));
}
}
// If the task state is CANCELING we need call noticeTaskExecutionServiceCancel().
if (ExecutionState.CANCELING.equals(executionState)) {
noticeTaskExecutionServiceCancel();
else if (ExecutionState.CANCELING.equals(executionState)) {
noticeTaskExecutionServiceCancel(3);
this.taskFuture.complete(new TaskExecutionState(taskGroupLocation, ExecutionState.CANCELED, null));
} else if (executionState.isEndState()) {
this.taskFuture.complete(new TaskExecutionState(taskGroupLocation, executionState, null));
}
return new PassiveCompletableFuture<>(this.taskFuture);
}

private boolean checkTaskGroupIsExecuting(TaskGroupLocation taskGroupLocation){
IMap<PipelineLocation, Map<TaskGroupLocation, SlotProfile>> ownedSlotProfilesIMap = nodeEngine.getHazelcastInstance().getMap(Constant.IMAP_OWNED_SLOT_PROFILES);
SlotProfile slotProfile = getOwnedSlotProfilesByTaskGroup(taskGroupLocation, ownedSlotProfilesIMap);
if (null != slotProfile){
Address worker = slotProfile.getWorker();
InvocationFuture<Object> invoke = nodeEngine.getOperationService().createInvocationBuilder(
SeaTunnelServer.SERVICE_NAME,
new CheckTaskGroupIsExecutingOperation(taskGroupLocation),
worker).invoke();
try {
return (Boolean) invoke.get();
} catch (InterruptedException | ExecutionException e) {
LOGGER.warning("Execution of CheckTaskGroupIsExecutingOperation failed, checkTaskGroupIsExecuting return false. ", e);
}
}
return false;
}

private SlotProfile getOwnedSlotProfilesByTaskGroup(TaskGroupLocation taskGroupLocation, IMap<PipelineLocation, Map<TaskGroupLocation, SlotProfile>> ownedSlotProfilesIMap) {
PipelineLocation pipelineLocation = taskGroupLocation.getPipelineLocation();
if (ownedSlotProfilesIMap.containsKey(pipelineLocation) &&
ownedSlotProfilesIMap.get(pipelineLocation).containsKey(taskGroupLocation)) {
return ownedSlotProfilesIMap.get(pipelineLocation).get(taskGroupLocation);
}
return null;
}

private void deployOnLocal(@NonNull SlotProfile slotProfile) {
deployInternal(taskGroupImmutableInformation -> {
SeaTunnelServer server = nodeEngine.getService(SeaTunnelServer.SERVICE_NAME);
Expand Down Expand Up @@ -333,15 +374,16 @@ public void cancel() {
updateTaskState(ExecutionState.DEPLOYING, ExecutionState.CANCELED)) {
taskFuture.complete(new TaskExecutionState(this.taskGroupLocation, ExecutionState.CANCELED, null));
} else if (updateTaskState(ExecutionState.RUNNING, ExecutionState.CANCELING)) {
noticeTaskExecutionServiceCancel();
noticeTaskExecutionServiceCancel(Integer.MAX_VALUE);
}
}

@SuppressWarnings("checkstyle:MagicNumber")
private void noticeTaskExecutionServiceCancel() {
private void noticeTaskExecutionServiceCancel(int tryTimes) {
int i = 0;
// In order not to generate uncontrolled tasks, We will try again until the taskFuture is completed
while (!taskFuture.isDone() && nodeEngine.getClusterService().getMember(getCurrentExecutionAddress()) != null) {
// If the cluster restart causes the number of nodes to change, it is meaningless to keep retrying
while (!taskFuture.isDone() && nodeEngine.getClusterService().getMember(getCurrentExecutionAddress()) != null && i < tryTimes) {
try {
i++;
LOGGER.info(
Expand Down Expand Up @@ -377,15 +419,6 @@ public ExecutionState getExecutionState() {

private void resetExecutionState() {
synchronized (this) {
ExecutionState executionState = getExecutionState();
if (!executionState.isEndState()) {
String message =
String.format("%s reset state failed, only end state can be reset, current is %s",
getTaskFullName(),
executionState);
LOGGER.severe(message);
throw new IllegalStateException(message);
}
updateStateTimestamps(ExecutionState.CREATED);
runningJobStateIMap.set(taskGroupLocation, ExecutionState.CREATED);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -378,7 +378,7 @@ private void forcePipelineFinish() {
public void restorePipelineState() {
// only need restore from RUNNING or CANCELING state
if (getPipelineState().ordinal() < PipelineStatus.RUNNING.ordinal()) {
restorePipeline();
cancelPipelineTasks();
} else if (PipelineStatus.CANCELING.equals(getPipelineState())) {
cancelPipelineTasks();
} else if (PipelineStatus.RUNNING.equals(getPipelineState())) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.seatunnel.engine.server.execution;

import org.apache.seatunnel.engine.server.dag.physical.PipelineLocation;

import lombok.AllArgsConstructor;
import lombok.Data;
import org.apache.commons.lang3.builder.EqualsBuilder;
Expand All @@ -33,6 +35,10 @@ public class TaskGroupLocation implements Serializable {

private final long taskGroupId;

public PipelineLocation getPipelineLocation() {
return new PipelineLocation(this.jobId, this.pipelineId);
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -176,13 +176,13 @@ public void init(long initializationTimestamp) throws Exception {
runningJobStateTimestampsIMap);
this.physicalPlan = planTuple.f0();
this.physicalPlan.setJobMaster(this);
this.initStateFuture();
this.checkpointManager = new CheckpointManager(
jobImmutableInformation.getJobId(),
nodeEngine,
this,
planTuple.f1(),
checkpointConfig);
this.initStateFuture();
}

// TODO replace it after ReadableConfig Support parse yaml format, then use only one config to read engine and env config.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,8 @@ public class TaskDataSerializerHook implements DataSerializerHook {

public static final int SOURCE_READER_EVENT_OPERATOR = 20;

public static final int CHECK_TASKGROUP_IS_EXECUTING = 21;

public static final int FACTORY_ID = FactoryIdHelper.getFactoryId(
SeaTunnelFactoryIdConstant.SEATUNNEL_TASK_DATA_SERIALIZER_FACTORY,
SeaTunnelFactoryIdConstant.SEATUNNEL_TASK_DATA_SERIALIZER_FACTORY_ID
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
/*
* 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.task.operation;

import org.apache.seatunnel.engine.server.SeaTunnelServer;
import org.apache.seatunnel.engine.server.execution.TaskGroupLocation;
import org.apache.seatunnel.engine.server.serializable.TaskDataSerializerHook;

import com.hazelcast.nio.ObjectDataInput;
import com.hazelcast.nio.ObjectDataOutput;
import com.hazelcast.nio.serialization.IdentifiedDataSerializable;
import com.hazelcast.spi.impl.operationservice.Operation;

import java.io.IOException;

public class CheckTaskGroupIsExecutingOperation extends Operation implements IdentifiedDataSerializable {

private TaskGroupLocation taskGroupLocation;
private Boolean response;

public CheckTaskGroupIsExecutingOperation() {
}

public CheckTaskGroupIsExecutingOperation(TaskGroupLocation taskGroupLocation) {
this.taskGroupLocation = taskGroupLocation;
}

@Override
public void run() {
SeaTunnelServer server = getService();
response = server.getTaskExecutionService().getExecutionContext(taskGroupLocation) != null;
}

@Override
protected void writeInternal(ObjectDataOutput out) throws IOException {
super.writeInternal(out);
out.writeObject(taskGroupLocation);
}

@Override
protected void readInternal(ObjectDataInput in) throws IOException {
super.readInternal(in);
taskGroupLocation = in.readObject();
}

@Override
public String getServiceName() {
return SeaTunnelServer.SERVICE_NAME;
}

@Override
public Object getResponse() {
return response;
}

@Override
public int getFactoryId() {
return TaskDataSerializerHook.FACTORY_ID;
}

@Override
public int getClassId() {
return TaskDataSerializerHook.CHECK_TASKGROUP_IS_EXECUTING;
}
}