Skip to content
Closed
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 @@ -91,6 +91,15 @@ List<WorkflowInstance> queryByWorkflowCodeVersionStatus(Long workflowDefinitionC
int workflowDefinitionVersion,
int[] states);

/**
* query workflow instance by workflowDefinitionCode and stateArray
*
* @param workflowDefinitionCode workflowDefinitionCode
* @param states states array
* @return workflow instance list
*/
List<WorkflowInstance> queryByWorkflowDefinitionCodeAndStatus(Long workflowDefinitionCode, int[] states);

List<String> queryNeedFailoverMasters();

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,6 +168,11 @@ public List<WorkflowInstance> queryByWorkflowCodeVersionStatus(Long workflowDefi
states);
}

@Override
public List<WorkflowInstance> queryByWorkflowDefinitionCodeAndStatus(Long workflowDefinitionCode, int[] states) {
return mybatisMapper.queryByWorkflowDefinitionCodeAndStatus(workflowDefinitionCode, states);
}

@Override
public List<String> queryNeedFailoverMasters() {
return mybatisMapper
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,132 @@
/*
* 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.dolphinscheduler.server.master.engine.command.handler;

import org.apache.dolphinscheduler.common.enums.CommandType;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;
import org.apache.dolphinscheduler.dao.repository.WorkflowInstanceDao;
import org.apache.dolphinscheduler.server.master.engine.graph.IWorkflowGraph;
import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowExecutionGraph;
import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowGraphTopologyLogicalVisitor;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnableBuilder;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteContext.WorkflowExecuteContextBuilder;

import java.util.Set;
import java.util.function.BiConsumer;

import lombok.extern.slf4j.Slf4j;

import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.context.ApplicationContext;
import org.springframework.stereotype.Component;

/**
* This handler used to handle {@link CommandType#RECOVER_SERIAL_WAIT}.
* Will recover a workflow instance from serial wait state to running state.
*/
@Slf4j
@Component
public class RecoverSerialWaitCommandHandler extends AbstractCommandHandler {

@Autowired
private WorkflowInstanceDao workflowInstanceDao;

@Autowired
private ApplicationContext applicationContext;

/**
* Generate the recover workflow instance from serial wait state.
* Will use the origin workflow instance, but will update the following fields:
* <ul>
* <li>state: from SERIAL_WAIT to RUNNING_EXECUTION</li>
* <li>command type</li>
* <li>start time</li>
* <li>restart time</li>
* </ul>
*/
@Override
protected void assembleWorkflowInstance(final WorkflowExecuteContextBuilder workflowExecuteContextBuilder) {
final Command command = workflowExecuteContextBuilder.getCommand();
final int workflowInstanceId = command.getWorkflowInstanceId();

log.info("Recovering workflow instance from SERIAL_WAIT state, instance id: {}", workflowInstanceId);

final WorkflowInstance workflowInstance = workflowInstanceDao.queryOptionalById(workflowInstanceId)
.orElseThrow(() -> new IllegalArgumentException("Cannot find WorkflowInstance:" + workflowInstanceId));

log.info("Found workflow instance: {} (id: {}), current state: {}",
workflowInstance.getName(), workflowInstance.getId(), workflowInstance.getState());

// Check if the workflow instance is in SERIAL_WAIT state
if (workflowInstance.getState() != WorkflowExecutionStatus.SERIAL_WAIT) {
log.warn("Workflow instance {} is not in SERIAL_WAIT state, current state: {}",
workflowInstance.getName(), workflowInstance.getState());
throw new IllegalStateException("Workflow instance is not in SERIAL_WAIT state");
}

workflowInstance.setStateWithDesc(WorkflowExecutionStatus.RUNNING_EXECUTION, command.getCommandType().name());
workflowInstance.setCommandType(command.getCommandType());
workflowInstanceDao.updateById(workflowInstance);

log.info("Successfully recovered workflow instance {} from SERIAL_WAIT to RUNNING_EXECUTION",
workflowInstance.getName());
workflowExecuteContextBuilder.setWorkflowInstance(workflowInstance);
}

@Override
protected void assembleWorkflowExecutionGraph(final WorkflowExecuteContextBuilder workflowExecuteContextBuilder) {
// For serial wait recovery, we need to create the workflow execution graph from the beginning
// This is similar to RunWorkflowCommandHandler
final IWorkflowGraph workflowGraph = workflowExecuteContextBuilder.getWorkflowGraph();
final WorkflowExecutionGraph workflowExecutionGraph = new WorkflowExecutionGraph();
final BiConsumer<String, Set<String>> taskExecutionRunnableCreator = (task, successors) -> {
final TaskExecutionRunnableBuilder taskExecutionRunnableBuilder =
TaskExecutionRunnableBuilder
.builder()
.workflowExecutionGraph(workflowExecutionGraph)
.workflowDefinition(workflowExecuteContextBuilder.getWorkflowDefinition())
.project(workflowExecuteContextBuilder.getProject())
.workflowInstance(workflowExecuteContextBuilder.getWorkflowInstance())
.taskDefinition(workflowGraph.getTaskNodeByName(task))
.workflowEventBus(workflowExecuteContextBuilder.getWorkflowEventBus())
.applicationContext(applicationContext)
.build();
workflowExecutionGraph.addNode(new TaskExecutionRunnable(taskExecutionRunnableBuilder));
workflowExecutionGraph.addEdge(task, successors);
};

final WorkflowGraphTopologyLogicalVisitor workflowGraphTopologyLogicalVisitor =
WorkflowGraphTopologyLogicalVisitor.builder()
.taskDependType(workflowExecuteContextBuilder.getWorkflowInstance().getTaskDependType())
.onWorkflowGraph(workflowGraph)
.fromTask(parseStartNodesFromWorkflowInstance(workflowExecuteContextBuilder))
.doVisitFunction(taskExecutionRunnableCreator)
.build();
workflowGraphTopologyLogicalVisitor.visit();

workflowExecuteContextBuilder.setWorkflowExecutionGraph(workflowExecutionGraph);
}

@Override
public CommandType commandType() {
return CommandType.RECOVER_SERIAL_WAIT;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,8 @@
import org.apache.dolphinscheduler.server.master.engine.graph.WorkflowGraphTopologyLogicalVisitor;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnable;
import org.apache.dolphinscheduler.server.master.engine.task.runnable.TaskExecutionRunnableBuilder;
import org.apache.dolphinscheduler.server.master.engine.workflow.strategy.WorkflowExecutionStrategyService;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteContext.WorkflowExecuteContextBuilder;
import org.apache.dolphinscheduler.service.expand.CuringParamsService;

import org.apache.commons.collections4.CollectionUtils;

Expand Down Expand Up @@ -70,7 +70,7 @@ public class RunWorkflowCommandHandler extends AbstractCommandHandler {
private ApplicationContext applicationContext;

@Autowired
private CuringParamsService curingParamsService;
private WorkflowExecutionStrategyService workflowExecutionStrategyService;

/**
* Will generate a new workflow instance based on the command.
Expand All @@ -80,7 +80,16 @@ protected void assembleWorkflowInstance(final WorkflowExecuteContextBuilder work
final WorkflowDefinition workflowDefinition = workflowExecuteContextBuilder.getWorkflowDefinition();
final Command command = workflowExecuteContextBuilder.getCommand();
final WorkflowInstance workflowInstance = workflowInstanceDao.queryById(command.getWorkflowInstanceId());
workflowInstance.setStateWithDesc(WorkflowExecutionStatus.RUNNING_EXECUTION, command.getCommandType().name());

// Apply execution strategy before setting the state
boolean shouldExecute = workflowExecutionStrategyService.checkAndApplyExecutionStrategy(
workflowDefinition, workflowInstance);

if (shouldExecute) {
workflowInstance.setStateWithDesc(WorkflowExecutionStatus.RUNNING_EXECUTION,
command.getCommandType().name());
}

workflowInstance.setHost(masterConfig.getMasterAddress());
workflowInstance.setCommandParam(command.getCommandParam());
workflowInstance.setGlobalParams(mergeCommandParamsWithWorkflowParams(command, workflowDefinition));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowFinalizeLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.lifecycle.event.WorkflowTopologyLogicalTransitionWithTaskFinishLifecycleEvent;
import org.apache.dolphinscheduler.server.master.engine.workflow.runnable.IWorkflowExecutionRunnable;
import org.apache.dolphinscheduler.server.master.engine.workflow.strategy.WorkflowExecutionStrategyService;
import org.apache.dolphinscheduler.server.master.utils.WorkflowInstanceUtils;
import org.apache.dolphinscheduler.service.alert.WorkflowAlertManager;

Expand Down Expand Up @@ -65,6 +66,9 @@ public abstract class AbstractWorkflowStateAction implements IWorkflowStateActio
@Autowired
protected WorkflowAlertManager workflowAlertManager;

@Autowired
protected WorkflowExecutionStrategyService workflowExecutionStrategyService;

/**
* Try to trigger the tasks if the trigger condition is met.
* <p> If all the given tasks trigger condition is not met then will try to emit workflow finish event.
Expand Down Expand Up @@ -196,6 +200,14 @@ protected void finalizeEventAction(final IWorkflowExecutionRunnable workflowExec
workflowEventBusCoordinator.unRegisterWorkflowEventBus(workflowExecutionRunnable);
workflowAlertManager.sendAlertWorkflowInstance(workflowExecutionRunnable.getWorkflowInstance());

// Check and wake up next serial waiting workflow instance
try {
workflowExecutionStrategyService
.checkAndWakeUpNextSerialInstance(workflowExecutionRunnable.getWorkflowInstance());
} catch (Exception e) {
log.warn("Failed to check and wake up next serial waiting workflow instance", e);
}

log.info("Successfully finalize WorkflowExecuteRunnable: {}", workflowExecutionRunnable.getName());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* 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.dolphinscheduler.server.master.engine.workflow.strategy;

import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
import org.apache.dolphinscheduler.dao.entity.WorkflowInstance;

public interface WorkflowExecutionStrategyService {

boolean checkAndApplyExecutionStrategy(WorkflowDefinition workflowDefinition,
WorkflowInstance workflowInstance);

boolean handleParallelStrategy(WorkflowDefinition workflowDefinition,

Check notice

Code scanning / CodeQL

Useless parameter Note

The parameter 'workflowDefinition' is never used.
WorkflowInstance workflowInstance);

boolean handleSerialWaitStrategy(WorkflowDefinition workflowDefinition,
WorkflowInstance workflowInstance);

boolean handleSerialDiscardStrategy(WorkflowDefinition workflowDefinition,
WorkflowInstance workflowInstance);

boolean handleSerialPriorityStrategy(WorkflowDefinition workflowDefinition,
WorkflowInstance workflowInstance);

void killWorkflowInstance(WorkflowInstance workflowInstance);

void checkAndWakeUpNextSerialInstance(WorkflowInstance completedInstance);

void wakeUpSerialWaitingInstance(WorkflowInstance waitingInstance);
}
Loading
Loading