Wenjun Ruan
3 months ago
committed by
GitHub
520 changed files with 23089 additions and 15969 deletions
@ -0,0 +1,96 @@
|
||||
/* |
||||
* 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.api.dto.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.common.enums.ComplementDependentMode; |
||||
import org.apache.dolphinscheduler.common.enums.ExecutionOrder; |
||||
import org.apache.dolphinscheduler.common.enums.FailureStrategy; |
||||
import org.apache.dolphinscheduler.common.enums.Flag; |
||||
import org.apache.dolphinscheduler.common.enums.Priority; |
||||
import org.apache.dolphinscheduler.common.enums.RunMode; |
||||
import org.apache.dolphinscheduler.common.enums.TaskDependType; |
||||
import org.apache.dolphinscheduler.common.enums.WarningType; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
|
||||
import lombok.AllArgsConstructor; |
||||
import lombok.Builder; |
||||
import lombok.Data; |
||||
import lombok.NoArgsConstructor; |
||||
|
||||
@Data |
||||
@Builder |
||||
@AllArgsConstructor |
||||
@NoArgsConstructor |
||||
public class WorkflowBackFillRequest { |
||||
|
||||
private User loginUser; |
||||
|
||||
private long workflowDefinitionCode; |
||||
|
||||
private String startNodes; |
||||
|
||||
private FailureStrategy failureStrategy; |
||||
|
||||
private TaskDependType taskDependType; |
||||
|
||||
private CommandType execType; |
||||
|
||||
private WarningType warningType; |
||||
|
||||
private Integer warningGroupId; |
||||
|
||||
private Priority workflowInstancePriority; |
||||
|
||||
private String workerGroup; |
||||
|
||||
private String tenantCode; |
||||
|
||||
private Long environmentCode; |
||||
|
||||
private String startParamList; |
||||
|
||||
private Flag dryRun; |
||||
|
||||
private Flag testFlag; |
||||
|
||||
private RunMode backfillRunMode; |
||||
|
||||
private BackfillTime backfillTime; |
||||
|
||||
private Integer expectedParallelismNumber; |
||||
|
||||
private ComplementDependentMode backfillDependentMode; |
||||
|
||||
private boolean allLevelDependent; |
||||
|
||||
private ExecutionOrder executionOrder; |
||||
|
||||
@Data |
||||
@Builder |
||||
@AllArgsConstructor |
||||
@NoArgsConstructor |
||||
public static class BackfillTime { |
||||
|
||||
private String complementStartDate; |
||||
private String complementEndDate; |
||||
private String complementScheduleDateList; |
||||
|
||||
} |
||||
|
||||
} |
@ -0,0 +1,66 @@
|
||||
/* |
||||
* 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.api.dto.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.common.enums.FailureStrategy; |
||||
import org.apache.dolphinscheduler.common.enums.Flag; |
||||
import org.apache.dolphinscheduler.common.enums.Priority; |
||||
import org.apache.dolphinscheduler.common.enums.TaskDependType; |
||||
import org.apache.dolphinscheduler.common.enums.WarningType; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
|
||||
import lombok.AllArgsConstructor; |
||||
import lombok.Builder; |
||||
import lombok.Data; |
||||
|
||||
@Data |
||||
@Builder |
||||
@AllArgsConstructor |
||||
public class WorkflowTriggerRequest { |
||||
|
||||
private User loginUser; |
||||
|
||||
private long workflowDefinitionCode; |
||||
|
||||
private String startNodes; |
||||
|
||||
private FailureStrategy failureStrategy; |
||||
|
||||
private TaskDependType taskDependType; |
||||
|
||||
private CommandType execType; |
||||
|
||||
private WarningType warningType; |
||||
|
||||
private Integer warningGroupId; |
||||
|
||||
private Priority workflowInstancePriority; |
||||
|
||||
private String workerGroup; |
||||
|
||||
private String tenantCode; |
||||
|
||||
private Long environmentCode; |
||||
|
||||
private String startParamList; |
||||
|
||||
private Flag dryRun; |
||||
|
||||
private Flag testFlag; |
||||
} |
@ -1,62 +0,0 @@
|
||||
/* |
||||
* 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.api.executor; |
||||
|
||||
import static com.google.common.base.Preconditions.checkNotNull; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
|
||||
import java.util.List; |
||||
import java.util.Map; |
||||
import java.util.function.Function; |
||||
import java.util.stream.Collectors; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
/** |
||||
* This is the main class for executing workflow/workflowInstance/tasks. |
||||
* <pre> |
||||
* ExecuteContext executeContext = ExecuteContext.builder() |
||||
* .processInstance(processInstance) |
||||
* .executeType(...) |
||||
* .build(); |
||||
* executeClient.execute(executeContext); |
||||
* </pre> |
||||
*/ |
||||
@Component |
||||
@SuppressWarnings("unchecked") |
||||
public class ExecuteClient { |
||||
|
||||
private final Map<ExecuteType, ExecuteFunctionBuilder> executorFunctionBuilderMap; |
||||
|
||||
public ExecuteClient(List<ExecuteFunctionBuilder> executeFunctionBuilders) { |
||||
executorFunctionBuilderMap = executeFunctionBuilders.stream() |
||||
.collect(Collectors.toMap(ExecuteFunctionBuilder::getExecuteType, Function.identity())); |
||||
} |
||||
|
||||
public ExecuteResult executeWorkflowInstance(ExecuteContext executeContext) throws ExecuteRuntimeException { |
||||
ExecuteFunctionBuilder<ExecuteRequest, ExecuteResult> executeFunctionBuilder = checkNotNull( |
||||
executorFunctionBuilderMap.get(executeContext.getExecuteType()), |
||||
String.format("The executeType: %s is not supported", executeContext.getExecuteType())); |
||||
|
||||
return executeFunctionBuilder.createWorkflowInstanceExecuteFunction(executeContext) |
||||
.thenCombine(executeFunctionBuilder.createWorkflowInstanceExecuteRequest(executeContext), |
||||
ExecuteFunction::execute) |
||||
.join(); |
||||
} |
||||
} |
@ -1,51 +0,0 @@
|
||||
/* |
||||
* 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.api.executor; |
||||
|
||||
import static com.google.common.base.Preconditions.checkNotNull; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
|
||||
import lombok.Data; |
||||
|
||||
// todo: to be interface
|
||||
@Data |
||||
public class ExecuteContext { |
||||
|
||||
private final ProcessInstance workflowInstance; |
||||
|
||||
private final ProcessDefinition workflowDefinition; |
||||
|
||||
private final User executeUser; |
||||
|
||||
private final ExecuteType executeType; |
||||
|
||||
public ExecuteContext(ProcessInstance workflowInstance, |
||||
ProcessDefinition workflowDefinition, |
||||
User executeUser, |
||||
ExecuteType executeType) { |
||||
this.workflowInstance = checkNotNull(workflowInstance, "workflowInstance cannot be null"); |
||||
this.workflowDefinition = checkNotNull(workflowDefinition, "workflowDefinition cannot be null"); |
||||
this.executeUser = checkNotNull(executeUser, "executeUser cannot be null"); |
||||
this.executeType = checkNotNull(executeType, "executeType cannot be null"); |
||||
} |
||||
|
||||
} |
@ -1,32 +0,0 @@
|
||||
/* |
||||
* 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.api.executor; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
|
||||
import java.util.concurrent.CompletableFuture; |
||||
|
||||
public interface ExecuteFunctionBuilder<Request extends ExecuteRequest, Result extends ExecuteResult> { |
||||
|
||||
CompletableFuture<ExecuteFunction<Request, Result>> createWorkflowInstanceExecuteFunction(ExecuteContext executeContext); |
||||
|
||||
CompletableFuture<Request> createWorkflowInstanceExecuteRequest(ExecuteContext executeContext); |
||||
|
||||
ExecuteType getExecuteType(); |
||||
|
||||
} |
@ -1,46 +0,0 @@
|
||||
/* |
||||
* 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.api.executor; |
||||
|
||||
// todo: implement from DolphinSchedulerRuntimeException
|
||||
public class ExecuteRuntimeException extends RuntimeException { |
||||
|
||||
private static final long serialVersionUID = 1L; |
||||
|
||||
private static final String EXECUTE_WORKFLOW_INSTANCE_ERROR = |
||||
"Execute workflow instance %s failed, execute type is %s"; |
||||
|
||||
public ExecuteRuntimeException(String message) { |
||||
super(message); |
||||
} |
||||
|
||||
public ExecuteRuntimeException(String message, Throwable cause) { |
||||
super(message, cause); |
||||
} |
||||
|
||||
public static ExecuteRuntimeException executeWorkflowInstanceError(ExecuteContext executeContext) { |
||||
return executeWorkflowInstanceError(executeContext, null); |
||||
} |
||||
|
||||
public static ExecuteRuntimeException executeWorkflowInstanceError(ExecuteContext executeContext, Throwable cause) { |
||||
return new ExecuteRuntimeException( |
||||
String.format(EXECUTE_WORKFLOW_INSTANCE_ERROR, executeContext.getWorkflowInstance().getName(), |
||||
executeContext.getExecuteType()), |
||||
cause); |
||||
} |
||||
} |
@ -0,0 +1,139 @@
|
||||
/* |
||||
* 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.api.executor.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.api.validator.workflow.BackfillWorkflowDTO; |
||||
import org.apache.dolphinscheduler.common.enums.ComplementDependentMode; |
||||
import org.apache.dolphinscheduler.common.enums.ExecutionOrder; |
||||
import org.apache.dolphinscheduler.common.enums.RunMode; |
||||
import org.apache.dolphinscheduler.common.utils.DateUtils; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.dao.entity.Command; |
||||
import org.apache.dolphinscheduler.dao.repository.CommandDao; |
||||
import org.apache.dolphinscheduler.extract.master.command.BackfillWorkflowCommandParam; |
||||
import org.apache.dolphinscheduler.service.process.ProcessService; |
||||
|
||||
import java.time.ZonedDateTime; |
||||
import java.util.Collections; |
||||
import java.util.Date; |
||||
import java.util.List; |
||||
import java.util.stream.Collectors; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
import com.google.common.collect.Lists; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class BackfillWorkflowExecutorDelegate implements IExecutorDelegate<BackfillWorkflowDTO, Void> { |
||||
|
||||
@Autowired |
||||
private CommandDao commandDao; |
||||
|
||||
@Autowired |
||||
private ProcessService processService; |
||||
|
||||
@Override |
||||
public Void execute(final BackfillWorkflowDTO backfillWorkflowDTO) { |
||||
// todo: directly call the master api to do backfill
|
||||
if (backfillWorkflowDTO.getBackfillParams().getRunMode() == RunMode.RUN_MODE_SERIAL) { |
||||
doSerialBackfillWorkflow(backfillWorkflowDTO); |
||||
} else { |
||||
doParallemBackfillWorkflow(backfillWorkflowDTO); |
||||
} |
||||
return null; |
||||
} |
||||
|
||||
private void doSerialBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO) { |
||||
final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams(); |
||||
final List<ZonedDateTime> backfillTimeList = backfillParams.getBackfillDateList(); |
||||
if (backfillParams.getExecutionOrder() == ExecutionOrder.DESC_ORDER) { |
||||
Collections.sort(backfillTimeList, Collections.reverseOrder()); |
||||
} else { |
||||
Collections.sort(backfillTimeList); |
||||
} |
||||
|
||||
final BackfillWorkflowCommandParam backfillWorkflowCommandParam = BackfillWorkflowCommandParam.builder() |
||||
.commandParams(backfillWorkflowDTO.getStartParamList()) |
||||
.startNodes(backfillWorkflowDTO.getStartNodes()) |
||||
.backfillTimeList(backfillTimeList.stream().map(DateUtils::dateToString).collect(Collectors.toList())) |
||||
.timeZone(DateUtils.getTimezone()) |
||||
.build(); |
||||
|
||||
doCreateCommand(backfillWorkflowDTO, backfillWorkflowCommandParam); |
||||
} |
||||
|
||||
private void doParallemBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO) { |
||||
final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams(); |
||||
Integer expectedParallelismNumber = backfillParams.getExpectedParallelismNumber(); |
||||
|
||||
List<ZonedDateTime> listDate = backfillParams.getBackfillDateList(); |
||||
if (expectedParallelismNumber != null) { |
||||
expectedParallelismNumber = Math.min(listDate.size(), expectedParallelismNumber); |
||||
} else { |
||||
expectedParallelismNumber = listDate.size(); |
||||
} |
||||
|
||||
log.info("In parallel mode, current expectedParallelismNumber:{}", expectedParallelismNumber); |
||||
for (List<ZonedDateTime> stringDate : Lists.partition(listDate, expectedParallelismNumber)) { |
||||
final BackfillWorkflowCommandParam backfillWorkflowCommandParam = BackfillWorkflowCommandParam.builder() |
||||
.commandParams(backfillWorkflowDTO.getStartParamList()) |
||||
.startNodes(backfillWorkflowDTO.getStartNodes()) |
||||
.backfillTimeList(stringDate.stream().map(DateUtils::dateToString).collect(Collectors.toList())) |
||||
.timeZone(DateUtils.getTimezone()) |
||||
.build(); |
||||
doCreateCommand(backfillWorkflowDTO, backfillWorkflowCommandParam); |
||||
} |
||||
} |
||||
|
||||
private void doCreateCommand(final BackfillWorkflowDTO backfillWorkflowDTO, |
||||
final BackfillWorkflowCommandParam backfillWorkflowCommandParam) { |
||||
List<String> backfillTimeList = backfillWorkflowCommandParam.getBackfillTimeList(); |
||||
final Command command = Command.builder() |
||||
.commandType(backfillWorkflowDTO.getExecType()) |
||||
.processDefinitionCode(backfillWorkflowDTO.getWorkflowDefinition().getCode()) |
||||
.processDefinitionVersion(backfillWorkflowDTO.getWorkflowDefinition().getVersion()) |
||||
.executorId(backfillWorkflowDTO.getLoginUser().getId()) |
||||
.scheduleTime(DateUtils.stringToDate(backfillTimeList.get(0))) |
||||
.commandParam(JSONUtils.toJsonString(backfillWorkflowCommandParam)) |
||||
.taskDependType(backfillWorkflowDTO.getTaskDependType()) |
||||
.failureStrategy(backfillWorkflowDTO.getFailureStrategy()) |
||||
.warningType(backfillWorkflowDTO.getWarningType()) |
||||
.warningGroupId(backfillWorkflowDTO.getWarningGroupId()) |
||||
.startTime(new Date()) |
||||
.processInstancePriority(backfillWorkflowDTO.getWorkflowInstancePriority()) |
||||
.updateTime(new Date()) |
||||
.workerGroup(backfillWorkflowDTO.getWorkerGroup()) |
||||
.tenantCode(backfillWorkflowDTO.getTenantCode()) |
||||
.dryRun(backfillWorkflowDTO.getDryRun().getCode()) |
||||
.testFlag(backfillWorkflowDTO.getTestFlag().getCode()) |
||||
.build(); |
||||
commandDao.insert(command); |
||||
final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams(); |
||||
if (backfillParams.getBackfillDependentMode() == ComplementDependentMode.ALL_DEPENDENT) { |
||||
doBackfillDependentWorkflow(backfillWorkflowCommandParam, command); |
||||
} |
||||
} |
||||
|
||||
private void doBackfillDependentWorkflow(final BackfillWorkflowCommandParam backfillWorkflowCommandParam, |
||||
final Command backfillCommand) { |
||||
} |
||||
} |
@ -0,0 +1,79 @@
|
||||
/* |
||||
* 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.api.executor.workflow; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class ExecutorClient { |
||||
|
||||
@Autowired |
||||
private TriggerWorkflowExecutorDelegate triggerWorkflowExecutorDelegate; |
||||
|
||||
@Autowired |
||||
private BackfillWorkflowExecutorDelegate backfillWorkflowExecutorDelegate; |
||||
|
||||
@Autowired |
||||
private RepeatRunningWorkflowInstanceExecutorDelegate repeatRunningWorkflowInstanceExecutorDelegate; |
||||
|
||||
@Autowired |
||||
private RecoverFailureTaskInstanceExecutorDelegate recoverFailureTaskInstanceExecutorDelegate; |
||||
|
||||
@Autowired |
||||
private RecoverSuspendedWorkflowInstanceExecutorDelegate recoverSuspendedWorkflowInstanceExecutorDelegate; |
||||
|
||||
@Autowired |
||||
private PauseWorkflowInstanceExecutorDelegate pauseWorkflowInstanceExecutorDelegate; |
||||
@Autowired |
||||
private StopWorkflowInstanceExecutorDelegate stopWorkflowInstanceExecutorDelegate; |
||||
|
||||
public TriggerWorkflowExecutorDelegate triggerWorkflowDefinition() { |
||||
return triggerWorkflowExecutorDelegate; |
||||
} |
||||
|
||||
public BackfillWorkflowExecutorDelegate backfillWorkflowDefinition() { |
||||
return backfillWorkflowExecutorDelegate; |
||||
} |
||||
|
||||
public RepeatRunningWorkflowInstanceExecutorDelegate.RepeatRunningWorkflowInstanceOperation repeatRunningWorkflowInstance() { |
||||
return new RepeatRunningWorkflowInstanceExecutorDelegate.RepeatRunningWorkflowInstanceOperation( |
||||
repeatRunningWorkflowInstanceExecutorDelegate); |
||||
} |
||||
|
||||
public RecoverFailureTaskInstanceExecutorDelegate.RecoverFailureTaskInstanceOperation recoverFailureTaskInstance() { |
||||
return new RecoverFailureTaskInstanceExecutorDelegate.RecoverFailureTaskInstanceOperation( |
||||
recoverFailureTaskInstanceExecutorDelegate); |
||||
} |
||||
|
||||
public RecoverSuspendedWorkflowInstanceExecutorDelegate.RecoverSuspendedWorkflowInstanceOperation recoverSuspendedWorkflowInstanceOperation() { |
||||
return new RecoverSuspendedWorkflowInstanceExecutorDelegate.RecoverSuspendedWorkflowInstanceOperation( |
||||
recoverSuspendedWorkflowInstanceExecutorDelegate); |
||||
} |
||||
|
||||
public PauseWorkflowInstanceExecutorDelegate.PauseWorkflowInstanceOperation pauseWorkflowInstance() { |
||||
return new PauseWorkflowInstanceExecutorDelegate.PauseWorkflowInstanceOperation( |
||||
pauseWorkflowInstanceExecutorDelegate); |
||||
} |
||||
|
||||
public StopWorkflowInstanceExecutorDelegate.StopWorkflowInstanceOperation stopWorkflowInstance() { |
||||
return new StopWorkflowInstanceExecutorDelegate.StopWorkflowInstanceOperation( |
||||
stopWorkflowInstanceExecutorDelegate); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,128 @@
|
||||
/* |
||||
* 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.api.executor.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.api.exceptions.ServiceException; |
||||
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; |
||||
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; |
||||
import org.apache.dolphinscheduler.extract.master.IWorkflowInstanceController; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseRequest; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstancePauseResponse; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class PauseWorkflowInstanceExecutorDelegate |
||||
implements |
||||
IExecutorDelegate<PauseWorkflowInstanceExecutorDelegate.PauseWorkflowInstanceOperation, Void> { |
||||
|
||||
@Autowired |
||||
private ProcessInstanceDao workflowInstanceDao; |
||||
|
||||
@Override |
||||
public Void execute(PauseWorkflowInstanceOperation workflowInstanceControlRequest) { |
||||
final ProcessInstance workflowInstance = workflowInstanceControlRequest.workflowInstance; |
||||
exceptionIfWorkflowInstanceCannotPause(workflowInstance); |
||||
if (ifWorkflowInstanceCanDirectPauseInDB(workflowInstance)) { |
||||
directPauseInDB(workflowInstance); |
||||
} else { |
||||
pauseInMaster(workflowInstance); |
||||
} |
||||
return null; |
||||
} |
||||
|
||||
private void exceptionIfWorkflowInstanceCannotPause(ProcessInstance workflowInstance) { |
||||
WorkflowExecutionStatus workflowInstanceState = workflowInstance.getState(); |
||||
if (workflowInstanceState.canPause()) { |
||||
return; |
||||
} |
||||
throw new ServiceException( |
||||
"The workflow instance: " + workflowInstance.getName() + " status is " + workflowInstanceState |
||||
+ ", can not pause"); |
||||
} |
||||
|
||||
private boolean ifWorkflowInstanceCanDirectPauseInDB(ProcessInstance workflowInstance) { |
||||
return workflowInstance.getState().canDirectPauseInDB(); |
||||
} |
||||
|
||||
private void directPauseInDB(ProcessInstance workflowInstance) { |
||||
workflowInstanceDao.updateWorkflowInstanceState( |
||||
workflowInstance.getId(), |
||||
workflowInstance.getState(), |
||||
WorkflowExecutionStatus.PAUSE); |
||||
log.info("Update workflow instance {} state from: {} to {} success", |
||||
workflowInstance.getName(), |
||||
workflowInstance.getState().name(), |
||||
WorkflowExecutionStatus.PAUSE.name()); |
||||
} |
||||
|
||||
private void pauseInMaster(ProcessInstance workflowInstance) { |
||||
try { |
||||
final WorkflowInstancePauseResponse pauseResponse = SingletonJdkDynamicRpcClientProxyFactory |
||||
.withService(IWorkflowInstanceController.class) |
||||
.withHost(workflowInstance.getHost()) |
||||
.pauseWorkflowInstance(new WorkflowInstancePauseRequest(workflowInstance.getId())); |
||||
|
||||
if (pauseResponse != null && pauseResponse.isSuccess()) { |
||||
log.info("WorkflowInstance: {} pause success", workflowInstance.getName()); |
||||
} else { |
||||
throw new ServiceException( |
||||
"WorkflowInstance: " + workflowInstance.getName() + " pause failed: " + pauseResponse); |
||||
} |
||||
} catch (ServiceException e) { |
||||
throw e; |
||||
} catch (Exception e) { |
||||
throw new ServiceException( |
||||
String.format("WorkflowInstance: %s pause failed", workflowInstance.getName()), e); |
||||
} |
||||
} |
||||
|
||||
public static class PauseWorkflowInstanceOperation { |
||||
|
||||
private final PauseWorkflowInstanceExecutorDelegate pauseWorkflowInstanceExecutorDelegate; |
||||
|
||||
private ProcessInstance workflowInstance; |
||||
|
||||
private User executeUser; |
||||
|
||||
public PauseWorkflowInstanceOperation(PauseWorkflowInstanceExecutorDelegate pauseWorkflowInstanceExecutorDelegate) { |
||||
this.pauseWorkflowInstanceExecutorDelegate = pauseWorkflowInstanceExecutorDelegate; |
||||
} |
||||
|
||||
public PauseWorkflowInstanceExecutorDelegate.PauseWorkflowInstanceOperation onWorkflowInstance(ProcessInstance workflowInstance) { |
||||
this.workflowInstance = workflowInstance; |
||||
return this; |
||||
} |
||||
|
||||
public PauseWorkflowInstanceExecutorDelegate.PauseWorkflowInstanceOperation byUser(User executeUser) { |
||||
this.executeUser = executeUser; |
||||
return this; |
||||
} |
||||
|
||||
public void execute() { |
||||
pauseWorkflowInstanceExecutorDelegate.execute(this); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,91 @@
|
||||
/* |
||||
* 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.api.executor.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.api.exceptions.ServiceException; |
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.dao.entity.Command; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
import org.apache.dolphinscheduler.dao.repository.CommandDao; |
||||
|
||||
import java.util.Date; |
||||
|
||||
import lombok.Getter; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class RecoverFailureTaskInstanceExecutorDelegate |
||||
implements |
||||
IExecutorDelegate<RecoverFailureTaskInstanceExecutorDelegate.RecoverFailureTaskInstanceOperation, Void> { |
||||
|
||||
@Autowired |
||||
private CommandDao commandDao; |
||||
|
||||
@Override |
||||
public Void execute(RecoverFailureTaskInstanceOperation recoverFailureTaskInstanceOperation) { |
||||
ProcessInstance workflowInstance = recoverFailureTaskInstanceOperation.getWorkflowInstance(); |
||||
if (!workflowInstance.getState().isFailure()) { |
||||
throw new ServiceException( |
||||
String.format("The workflow instance: %s status is %s, can not be recovered", |
||||
workflowInstance.getName(), workflowInstance.getState())); |
||||
} |
||||
|
||||
Command command = Command.builder() |
||||
.commandType(CommandType.START_FAILURE_TASK_PROCESS) |
||||
.processDefinitionCode(workflowInstance.getProcessDefinitionCode()) |
||||
.processDefinitionVersion(workflowInstance.getProcessDefinitionVersion()) |
||||
.processInstanceId(workflowInstance.getId()) |
||||
.executorId(recoverFailureTaskInstanceOperation.getExecuteUser().getId()) |
||||
.startTime(new Date()) |
||||
.updateTime(new Date()) |
||||
.build(); |
||||
commandDao.insert(command); |
||||
return null; |
||||
} |
||||
|
||||
@Getter |
||||
public static class RecoverFailureTaskInstanceOperation { |
||||
|
||||
private final RecoverFailureTaskInstanceExecutorDelegate recoverFailureTaskInstanceExecutorDelegate; |
||||
|
||||
private ProcessInstance workflowInstance; |
||||
|
||||
private User executeUser; |
||||
|
||||
public RecoverFailureTaskInstanceOperation(RecoverFailureTaskInstanceExecutorDelegate recoverFailureTaskInstanceExecutorDelegate) { |
||||
this.recoverFailureTaskInstanceExecutorDelegate = recoverFailureTaskInstanceExecutorDelegate; |
||||
} |
||||
|
||||
public RecoverFailureTaskInstanceOperation onWorkflowInstance(ProcessInstance workflowInstance) { |
||||
this.workflowInstance = workflowInstance; |
||||
return this; |
||||
} |
||||
|
||||
public RecoverFailureTaskInstanceOperation byUser(User executeUser) { |
||||
this.executeUser = executeUser; |
||||
return this; |
||||
} |
||||
|
||||
public void execute() { |
||||
recoverFailureTaskInstanceExecutorDelegate.execute(this); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,87 @@
|
||||
/* |
||||
* 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.api.executor.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.api.exceptions.ServiceException; |
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.dao.entity.Command; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
import org.apache.dolphinscheduler.dao.repository.CommandDao; |
||||
|
||||
import java.util.Date; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class RecoverSuspendedWorkflowInstanceExecutorDelegate |
||||
implements |
||||
IExecutorDelegate<RecoverSuspendedWorkflowInstanceExecutorDelegate.RecoverSuspendedWorkflowInstanceOperation, Void> { |
||||
|
||||
@Autowired |
||||
private CommandDao commandDao; |
||||
|
||||
@Override |
||||
public Void execute(RecoverSuspendedWorkflowInstanceOperation workflowInstanceControlRequest) { |
||||
final ProcessInstance workflowInstance = workflowInstanceControlRequest.workflowInstance; |
||||
if (!workflowInstance.getState().isPause() && !workflowInstance.getState().isStop()) { |
||||
throw new ServiceException( |
||||
String.format("The workflow instance: %s state is %s, cannot recovery", workflowInstance.getName(), |
||||
workflowInstance.getState())); |
||||
} |
||||
final Command command = Command.builder() |
||||
.commandType(CommandType.RECOVER_SUSPENDED_PROCESS) |
||||
.processDefinitionCode(workflowInstance.getProcessDefinitionCode()) |
||||
.processDefinitionVersion(workflowInstance.getProcessDefinitionVersion()) |
||||
.processInstanceId(workflowInstance.getId()) |
||||
.executorId(workflowInstanceControlRequest.executeUser.getId()) |
||||
.startTime(new Date()) |
||||
.updateTime(new Date()) |
||||
.build(); |
||||
commandDao.insert(command); |
||||
return null; |
||||
} |
||||
|
||||
public static class RecoverSuspendedWorkflowInstanceOperation { |
||||
|
||||
private final RecoverSuspendedWorkflowInstanceExecutorDelegate recoverSuspendedWorkflowInstanceExecutorDelegate; |
||||
|
||||
private ProcessInstance workflowInstance; |
||||
|
||||
private User executeUser; |
||||
|
||||
public RecoverSuspendedWorkflowInstanceOperation(RecoverSuspendedWorkflowInstanceExecutorDelegate recoverSuspendedWorkflowInstanceExecutorDelegate) { |
||||
this.recoverSuspendedWorkflowInstanceExecutorDelegate = recoverSuspendedWorkflowInstanceExecutorDelegate; |
||||
} |
||||
|
||||
public RecoverSuspendedWorkflowInstanceExecutorDelegate.RecoverSuspendedWorkflowInstanceOperation onWorkflowInstance(ProcessInstance workflowInstance) { |
||||
this.workflowInstance = workflowInstance; |
||||
return this; |
||||
} |
||||
|
||||
public RecoverSuspendedWorkflowInstanceExecutorDelegate.RecoverSuspendedWorkflowInstanceOperation byUser(User executeUser) { |
||||
this.executeUser = executeUser; |
||||
return this; |
||||
} |
||||
|
||||
public void execute() { |
||||
recoverSuspendedWorkflowInstanceExecutorDelegate.execute(this); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,88 @@
|
||||
/* |
||||
* 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.api.executor.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.api.exceptions.ServiceException; |
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.dao.entity.Command; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
import org.apache.dolphinscheduler.dao.repository.CommandDao; |
||||
|
||||
import java.util.Date; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class RepeatRunningWorkflowInstanceExecutorDelegate |
||||
implements |
||||
IExecutorDelegate<RepeatRunningWorkflowInstanceExecutorDelegate.RepeatRunningWorkflowInstanceOperation, Void> { |
||||
|
||||
@Autowired |
||||
private CommandDao commandDao; |
||||
|
||||
@Override |
||||
public Void execute(RepeatRunningWorkflowInstanceOperation workflowInstanceControlRequest) { |
||||
final ProcessInstance workflowInstance = workflowInstanceControlRequest.workflowInstance; |
||||
if (workflowInstance.getState() == null || !workflowInstance.getState().isFinished()) { |
||||
throw new ServiceException( |
||||
String.format("The workflow instance: %s status is %s, cannot repeat running", |
||||
workflowInstance.getName(), workflowInstance.getState())); |
||||
} |
||||
Command command = Command.builder() |
||||
.commandType(CommandType.REPEAT_RUNNING) |
||||
.processInstanceId(workflowInstance.getId()) |
||||
.processDefinitionCode(workflowInstance.getProcessDefinitionCode()) |
||||
.processDefinitionVersion(workflowInstance.getProcessDefinitionVersion()) |
||||
.executorId(workflowInstanceControlRequest.executeUser.getId()) |
||||
.startTime(new Date()) |
||||
.updateTime(new Date()) |
||||
.build(); |
||||
commandDao.insert(command); |
||||
return null; |
||||
} |
||||
|
||||
public static class RepeatRunningWorkflowInstanceOperation { |
||||
|
||||
private final RepeatRunningWorkflowInstanceExecutorDelegate repeatRunningWorkflowInstanceExecutorDelegate; |
||||
|
||||
private ProcessInstance workflowInstance; |
||||
|
||||
private User executeUser; |
||||
|
||||
public RepeatRunningWorkflowInstanceOperation(RepeatRunningWorkflowInstanceExecutorDelegate repeatRunningWorkflowInstanceExecutorDelegate) { |
||||
this.repeatRunningWorkflowInstanceExecutorDelegate = repeatRunningWorkflowInstanceExecutorDelegate; |
||||
} |
||||
|
||||
public RepeatRunningWorkflowInstanceOperation onWorkflowInstance(ProcessInstance workflowInstance) { |
||||
this.workflowInstance = workflowInstance; |
||||
return this; |
||||
} |
||||
|
||||
public RepeatRunningWorkflowInstanceOperation byUser(User executeUser) { |
||||
this.executeUser = executeUser; |
||||
return this; |
||||
} |
||||
|
||||
public void execute() { |
||||
repeatRunningWorkflowInstanceExecutorDelegate.execute(this); |
||||
} |
||||
} |
||||
|
||||
} |
@ -0,0 +1,129 @@
|
||||
/* |
||||
* 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.api.executor.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.api.exceptions.ServiceException; |
||||
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; |
||||
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; |
||||
import org.apache.dolphinscheduler.extract.master.IWorkflowInstanceController; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopRequest; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStopResponse; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class StopWorkflowInstanceExecutorDelegate |
||||
implements |
||||
IExecutorDelegate<StopWorkflowInstanceExecutorDelegate.StopWorkflowInstanceOperation, Void> { |
||||
|
||||
@Autowired |
||||
private ProcessInstanceDao workflowInstanceDao; |
||||
|
||||
@Override |
||||
public Void execute(StopWorkflowInstanceOperation workflowInstanceControlRequest) { |
||||
final ProcessInstance workflowInstance = workflowInstanceControlRequest.workflowInstance; |
||||
exceptionIfWorkflowInstanceCannotStop(workflowInstance); |
||||
|
||||
if (ifWorkflowInstanceCanDirectStopInDB(workflowInstance)) { |
||||
directStopInDB(workflowInstance); |
||||
} else { |
||||
stopInMaster(workflowInstance); |
||||
} |
||||
return null; |
||||
} |
||||
|
||||
void exceptionIfWorkflowInstanceCannotStop(ProcessInstance workflowInstance) { |
||||
final WorkflowExecutionStatus workflowInstanceState = workflowInstance.getState(); |
||||
if (workflowInstanceState.canStop()) { |
||||
return; |
||||
} |
||||
throw new ServiceException( |
||||
"The workflow instance: " + workflowInstance.getName() + " status is " + workflowInstanceState |
||||
+ ", can not stop"); |
||||
} |
||||
|
||||
boolean ifWorkflowInstanceCanDirectStopInDB(ProcessInstance workflowInstance) { |
||||
return workflowInstance.getState().canDirectStopInDB(); |
||||
} |
||||
|
||||
void directStopInDB(ProcessInstance workflowInstance) { |
||||
workflowInstanceDao.updateWorkflowInstanceState( |
||||
workflowInstance.getId(), |
||||
workflowInstance.getState(), |
||||
WorkflowExecutionStatus.STOP); |
||||
log.info("Update workflow instance {} state from: {} to {} success", |
||||
workflowInstance.getName(), |
||||
workflowInstance.getState().name(), |
||||
WorkflowExecutionStatus.STOP.name()); |
||||
} |
||||
|
||||
void stopInMaster(ProcessInstance workflowInstance) { |
||||
try { |
||||
final WorkflowInstanceStopResponse stopResponse = SingletonJdkDynamicRpcClientProxyFactory |
||||
.withService(IWorkflowInstanceController.class) |
||||
.withHost(workflowInstance.getHost()) |
||||
.stopWorkflowInstance(new WorkflowInstanceStopRequest(workflowInstance.getId())); |
||||
|
||||
if (stopResponse != null && stopResponse.isSuccess()) { |
||||
log.info("WorkflowInstance: {} stop success", workflowInstance.getName()); |
||||
} else { |
||||
throw new ServiceException( |
||||
"WorkflowInstance: " + workflowInstance.getName() + " stop failed: " + stopResponse); |
||||
} |
||||
} catch (ServiceException e) { |
||||
throw e; |
||||
} catch (Exception e) { |
||||
throw new ServiceException( |
||||
String.format("WorkflowInstance: %s stop failed", workflowInstance.getName()), e); |
||||
} |
||||
} |
||||
|
||||
public static class StopWorkflowInstanceOperation { |
||||
|
||||
private final StopWorkflowInstanceExecutorDelegate stopWorkflowInstanceExecutorDelegate; |
||||
|
||||
private ProcessInstance workflowInstance; |
||||
|
||||
private User executeUser; |
||||
|
||||
public StopWorkflowInstanceOperation(StopWorkflowInstanceExecutorDelegate stopWorkflowInstanceExecutorDelegate) { |
||||
this.stopWorkflowInstanceExecutorDelegate = stopWorkflowInstanceExecutorDelegate; |
||||
} |
||||
|
||||
public StopWorkflowInstanceExecutorDelegate.StopWorkflowInstanceOperation onWorkflowInstance(ProcessInstance workflowInstance) { |
||||
this.workflowInstance = workflowInstance; |
||||
return this; |
||||
} |
||||
|
||||
public StopWorkflowInstanceExecutorDelegate.StopWorkflowInstanceOperation byUser(User executeUser) { |
||||
this.executeUser = executeUser; |
||||
return this; |
||||
} |
||||
|
||||
public void execute() { |
||||
stopWorkflowInstanceExecutorDelegate.execute(this); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,71 @@
|
||||
/* |
||||
* 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.api.executor.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.api.validator.workflow.TriggerWorkflowDTO; |
||||
import org.apache.dolphinscheduler.common.utils.DateUtils; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.dao.entity.Command; |
||||
import org.apache.dolphinscheduler.dao.repository.CommandDao; |
||||
import org.apache.dolphinscheduler.extract.master.command.RunWorkflowCommandParam; |
||||
import org.apache.dolphinscheduler.service.process.TriggerRelationService; |
||||
|
||||
import java.util.Date; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class TriggerWorkflowExecutorDelegate implements IExecutorDelegate<TriggerWorkflowDTO, Void> { |
||||
|
||||
@Autowired |
||||
private CommandDao commandDao; |
||||
|
||||
@Autowired |
||||
private TriggerRelationService triggerRelationService; |
||||
|
||||
@Override |
||||
public Void execute(TriggerWorkflowDTO triggerWorkflowDTO) { |
||||
final RunWorkflowCommandParam runWorkflowCommandParam = |
||||
RunWorkflowCommandParam.builder() |
||||
.commandParams(triggerWorkflowDTO.getStartParamList()) |
||||
.startNodes(triggerWorkflowDTO.getStartNodes()) |
||||
.timeZone(DateUtils.getTimezone()) |
||||
.build(); |
||||
final Command command = Command.builder() |
||||
.commandType(triggerWorkflowDTO.getExecType()) |
||||
.processDefinitionCode(triggerWorkflowDTO.getWorkflowDefinition().getCode()) |
||||
.processDefinitionVersion(triggerWorkflowDTO.getWorkflowDefinition().getVersion()) |
||||
.executorId(triggerWorkflowDTO.getLoginUser().getId()) |
||||
.commandParam(JSONUtils.toJsonString(runWorkflowCommandParam)) |
||||
.taskDependType(triggerWorkflowDTO.getTaskDependType()) |
||||
.failureStrategy(triggerWorkflowDTO.getFailureStrategy()) |
||||
.warningType(triggerWorkflowDTO.getWarningType()) |
||||
.warningGroupId(triggerWorkflowDTO.getWarningGroupId()) |
||||
.startTime(new Date()) |
||||
.processInstancePriority(triggerWorkflowDTO.getWorkflowInstancePriority()) |
||||
.updateTime(new Date()) |
||||
.workerGroup(triggerWorkflowDTO.getWorkerGroup()) |
||||
.tenantCode(triggerWorkflowDTO.getTenantCode()) |
||||
.dryRun(triggerWorkflowDTO.getDryRun().getCode()) |
||||
.testFlag(triggerWorkflowDTO.getTestFlag().getCode()) |
||||
.build(); |
||||
commandDao.insert(command); |
||||
return null; |
||||
} |
||||
} |
@ -1,64 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.failure.recovery; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunction; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteRuntimeException; |
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.dao.entity.Command; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.service.command.CommandService; |
||||
|
||||
public class FailureRecoveryExecuteFunction implements ExecuteFunction<FailureRecoveryRequest, FailureRecoveryResult> { |
||||
|
||||
private final CommandService commandService; |
||||
|
||||
public FailureRecoveryExecuteFunction(CommandService commandService) { |
||||
this.commandService = commandService; |
||||
} |
||||
|
||||
@Override |
||||
public FailureRecoveryResult execute(FailureRecoveryRequest request) throws ExecuteRuntimeException { |
||||
ProcessInstance workflowInstance = request.getWorkflowInstance(); |
||||
if (!workflowInstance.getState().isFailure()) { |
||||
throw new ExecuteRuntimeException( |
||||
String.format("The workflow instance: %s status is %s, can not be recovered", |
||||
workflowInstance.getName(), workflowInstance.getState())); |
||||
} |
||||
|
||||
Command command = Command.builder() |
||||
.commandType(CommandType.START_FAILURE_TASK_PROCESS) |
||||
.processDefinitionCode(workflowInstance.getProcessDefinitionCode()) |
||||
.processDefinitionVersion(workflowInstance.getProcessDefinitionVersion()) |
||||
.processInstanceId(workflowInstance.getId()) |
||||
.executorId(request.getExecuteUser().getId()) |
||||
.testFlag(workflowInstance.getTestFlag()) |
||||
.build(); |
||||
if (commandService.createCommand(command) <= 0) { |
||||
throw new ExecuteRuntimeException( |
||||
"Failure recovery workflow instance failed, due to insert command to db failed"); |
||||
} |
||||
return new FailureRecoveryResult(command.getId()); |
||||
} |
||||
|
||||
@Override |
||||
public ExecuteType getExecuteType() { |
||||
return FailureRecoveryExecuteFunctionBuilder.EXECUTE_TYPE; |
||||
} |
||||
} |
@ -1,59 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.failure.recovery; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteContext; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunction; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunctionBuilder; |
||||
import org.apache.dolphinscheduler.service.command.CommandService; |
||||
|
||||
import java.util.concurrent.CompletableFuture; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class FailureRecoveryExecuteFunctionBuilder |
||||
implements |
||||
ExecuteFunctionBuilder<FailureRecoveryRequest, FailureRecoveryResult> { |
||||
|
||||
public static final ExecuteType EXECUTE_TYPE = ExecuteType.START_FAILURE_TASK_PROCESS; |
||||
|
||||
@Autowired |
||||
private CommandService commandService; |
||||
|
||||
@Override |
||||
public CompletableFuture<ExecuteFunction<FailureRecoveryRequest, FailureRecoveryResult>> createWorkflowInstanceExecuteFunction(ExecuteContext executeContext) { |
||||
return CompletableFuture.completedFuture(new FailureRecoveryExecuteFunction(commandService)); |
||||
} |
||||
|
||||
@Override |
||||
public CompletableFuture<FailureRecoveryRequest> createWorkflowInstanceExecuteRequest(ExecuteContext executeContext) { |
||||
return CompletableFuture.completedFuture( |
||||
new FailureRecoveryRequest( |
||||
executeContext.getWorkflowInstance(), |
||||
executeContext.getWorkflowDefinition(), |
||||
executeContext.getExecuteUser())); |
||||
} |
||||
|
||||
@Override |
||||
public ExecuteType getExecuteType() { |
||||
return EXECUTE_TYPE; |
||||
} |
||||
} |
@ -1,79 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.pause.pause; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunction; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteRuntimeException; |
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; |
||||
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; |
||||
import org.apache.dolphinscheduler.extract.master.ITaskInstanceExecutionEventListener; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStateChangeEvent; |
||||
|
||||
public class PauseExecuteFunction implements ExecuteFunction<PauseExecuteRequest, PauseExecuteResult> { |
||||
|
||||
private final ProcessInstanceDao processInstanceDao; |
||||
|
||||
public PauseExecuteFunction(ProcessInstanceDao processInstanceDao) { |
||||
this.processInstanceDao = processInstanceDao; |
||||
} |
||||
|
||||
@Override |
||||
public PauseExecuteResult execute(PauseExecuteRequest request) throws ExecuteRuntimeException { |
||||
ProcessInstance workflowInstance = request.getWorkflowInstance(); |
||||
if (!workflowInstance.getState().isRunning()) { |
||||
throw new ExecuteRuntimeException( |
||||
String.format("The workflow instance: %s status is %s, can not pause", workflowInstance.getName(), |
||||
workflowInstance.getState())); |
||||
} |
||||
workflowInstance.setCommandType(CommandType.PAUSE); |
||||
workflowInstance.addHistoryCmd(CommandType.PAUSE); |
||||
workflowInstance.setStateWithDesc(WorkflowExecutionStatus.READY_PAUSE, |
||||
CommandType.PAUSE.getDescp() + " by " + request.getExecuteUser().getUserName()); |
||||
|
||||
if (!processInstanceDao.updateById(workflowInstance)) { |
||||
throw new ExecuteRuntimeException( |
||||
String.format( |
||||
"The workflow instance: %s pause failed, due to update the workflow instance status in DB failed", |
||||
workflowInstance.getName())); |
||||
} |
||||
try { |
||||
// todo: direct call the workflow instance pause method
|
||||
ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = |
||||
SingletonJdkDynamicRpcClientProxyFactory |
||||
.getProxyClient(workflowInstance.getHost(), ITaskInstanceExecutionEventListener.class); |
||||
iTaskInstanceExecutionEventListener.onWorkflowInstanceInstanceStateChange( |
||||
new WorkflowInstanceStateChangeEvent( |
||||
workflowInstance.getId(), 0, workflowInstance.getState(), workflowInstance.getId(), 0)); |
||||
} catch (Exception e) { |
||||
throw new ExecuteRuntimeException( |
||||
String.format( |
||||
"WorkflowInstance: %s pause failed", workflowInstance.getName()), |
||||
e); |
||||
} |
||||
return new PauseExecuteResult(workflowInstance); |
||||
} |
||||
|
||||
@Override |
||||
public ExecuteType getExecuteType() { |
||||
return PauseExecuteFunctionBuilder.EXECUTE_TYPE; |
||||
} |
||||
} |
@ -1,57 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.pause.pause; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteContext; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunction; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunctionBuilder; |
||||
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; |
||||
|
||||
import java.util.concurrent.CompletableFuture; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class PauseExecuteFunctionBuilder implements ExecuteFunctionBuilder<PauseExecuteRequest, PauseExecuteResult> { |
||||
|
||||
public static final ExecuteType EXECUTE_TYPE = ExecuteType.PAUSE; |
||||
|
||||
@Autowired |
||||
private ProcessInstanceDao processInstanceDao; |
||||
|
||||
@Override |
||||
public CompletableFuture<ExecuteFunction<PauseExecuteRequest, PauseExecuteResult>> createWorkflowInstanceExecuteFunction(ExecuteContext executeContext) { |
||||
return CompletableFuture.completedFuture(new PauseExecuteFunction(processInstanceDao)); |
||||
} |
||||
|
||||
@Override |
||||
public CompletableFuture<PauseExecuteRequest> createWorkflowInstanceExecuteRequest(ExecuteContext executeContext) { |
||||
return CompletableFuture.completedFuture( |
||||
new PauseExecuteRequest( |
||||
executeContext.getWorkflowDefinition(), |
||||
executeContext.getWorkflowInstance(), |
||||
executeContext.getExecuteUser())); |
||||
} |
||||
|
||||
@Override |
||||
public ExecuteType getExecuteType() { |
||||
return EXECUTE_TYPE; |
||||
} |
||||
} |
@ -1,35 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.pause.pause; |
||||
|
||||
import org.apache.dolphinscheduler.api.executor.ExecuteRequest; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
|
||||
import lombok.AllArgsConstructor; |
||||
import lombok.Data; |
||||
|
||||
@Data |
||||
@AllArgsConstructor |
||||
public class PauseExecuteRequest implements ExecuteRequest { |
||||
|
||||
private final ProcessDefinition processDefinition; |
||||
private final ProcessInstance workflowInstance; |
||||
private final User executeUser; |
||||
} |
@ -1,29 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.pause.pause; |
||||
|
||||
import org.apache.dolphinscheduler.api.executor.ExecuteResult; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
|
||||
import lombok.AllArgsConstructor; |
||||
|
||||
@AllArgsConstructor |
||||
public class PauseExecuteResult implements ExecuteResult { |
||||
|
||||
private final ProcessInstance processInstance; |
||||
} |
@ -1,78 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.pause.recover; |
||||
|
||||
import static org.apache.dolphinscheduler.common.constants.CommandKeyConstants.CMD_PARAM_RECOVER_PROCESS_ID_STRING; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunction; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteRuntimeException; |
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.dao.entity.Command; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.service.command.CommandService; |
||||
|
||||
import java.util.Map; |
||||
|
||||
import com.google.common.collect.ImmutableMap; |
||||
|
||||
public class RecoverExecuteFunction implements ExecuteFunction<RecoverExecuteRequest, RecoverExecuteResult> { |
||||
|
||||
private final CommandService commandService; |
||||
|
||||
public RecoverExecuteFunction(CommandService commandService) { |
||||
this.commandService = commandService; |
||||
} |
||||
|
||||
@Override |
||||
public RecoverExecuteResult execute(RecoverExecuteRequest request) throws ExecuteRuntimeException { |
||||
ProcessInstance workflowInstance = request.getWorkflowInstance(); |
||||
if (!(workflowInstance.getState().isPause() || workflowInstance.getState().isStop())) { |
||||
throw new ExecuteRuntimeException( |
||||
String.format("The workflow instance: %s state is %s, cannot recovery", workflowInstance.getName(), |
||||
workflowInstance.getState())); |
||||
} |
||||
Command command = Command.builder() |
||||
.commandType(CommandType.RECOVER_SUSPENDED_PROCESS) |
||||
.processDefinitionCode(workflowInstance.getProcessDefinitionCode()) |
||||
.processDefinitionVersion(workflowInstance.getProcessDefinitionVersion()) |
||||
.processInstanceId(workflowInstance.getId()) |
||||
.commandParam(JSONUtils.toJsonString(createCommandParam(workflowInstance))) |
||||
.executorId(request.getExecuteUser().getId()) |
||||
.testFlag(workflowInstance.getTestFlag()) |
||||
.build(); |
||||
if (commandService.createCommand(command) <= 0) { |
||||
throw new ExecuteRuntimeException( |
||||
String.format("Recovery workflow instance: %s failed, due to insert command to db failed", |
||||
workflowInstance.getName())); |
||||
} |
||||
return new RecoverExecuteResult(command); |
||||
} |
||||
|
||||
private Map<String, Object> createCommandParam(ProcessInstance workflowInstance) { |
||||
return new ImmutableMap.Builder<String, Object>() |
||||
.put(CMD_PARAM_RECOVER_PROCESS_ID_STRING, workflowInstance.getId()) |
||||
.build(); |
||||
} |
||||
|
||||
@Override |
||||
public ExecuteType getExecuteType() { |
||||
return RecoverExecuteFunctionBuilder.EXECUTE_TYPE; |
||||
} |
||||
} |
@ -1,60 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.pause.recover; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteContext; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunction; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunctionBuilder; |
||||
import org.apache.dolphinscheduler.service.command.CommandService; |
||||
|
||||
import java.util.concurrent.CompletableFuture; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class RecoverExecuteFunctionBuilder |
||||
implements |
||||
ExecuteFunctionBuilder<RecoverExecuteRequest, RecoverExecuteResult> { |
||||
|
||||
public static final ExecuteType EXECUTE_TYPE = ExecuteType.RECOVER_SUSPENDED_PROCESS; |
||||
|
||||
@Autowired |
||||
private CommandService commandService; |
||||
|
||||
@Override |
||||
public CompletableFuture<ExecuteFunction<RecoverExecuteRequest, RecoverExecuteResult>> createWorkflowInstanceExecuteFunction(ExecuteContext executeContext) { |
||||
return CompletableFuture.completedFuture( |
||||
new RecoverExecuteFunction(commandService)); |
||||
} |
||||
|
||||
@Override |
||||
public CompletableFuture<RecoverExecuteRequest> createWorkflowInstanceExecuteRequest(ExecuteContext executeContext) { |
||||
return CompletableFuture.completedFuture( |
||||
new RecoverExecuteRequest( |
||||
executeContext.getWorkflowInstance(), |
||||
executeContext.getWorkflowDefinition(), |
||||
executeContext.getExecuteUser())); |
||||
} |
||||
|
||||
@Override |
||||
public ExecuteType getExecuteType() { |
||||
return EXECUTE_TYPE; |
||||
} |
||||
} |
@ -1,93 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.rerun; |
||||
|
||||
import static com.google.common.base.Preconditions.checkNotNull; |
||||
import static org.apache.dolphinscheduler.common.constants.CommandKeyConstants.CMD_PARAM_RECOVER_PROCESS_ID_STRING; |
||||
import static org.apache.dolphinscheduler.common.constants.CommandKeyConstants.CMD_PARAM_START_PARAMS; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunction; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteRuntimeException; |
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
import org.apache.dolphinscheduler.dao.entity.Command; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.service.command.CommandService; |
||||
|
||||
import org.apache.commons.collections4.MapUtils; |
||||
|
||||
import java.util.HashMap; |
||||
import java.util.Map; |
||||
import java.util.Optional; |
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference; |
||||
|
||||
public class RepeatRunningExecuteFunction implements ExecuteFunction<RepeatRunningRequest, RepeatRunningResult> { |
||||
|
||||
private final CommandService commandService; |
||||
|
||||
public RepeatRunningExecuteFunction(CommandService commandService) { |
||||
this.commandService = commandService; |
||||
} |
||||
|
||||
@Override |
||||
public RepeatRunningResult execute(RepeatRunningRequest request) throws ExecuteRuntimeException { |
||||
checkNotNull(request, "request cannot be null"); |
||||
// todo: check workflow definition valid? or we don't need to do this check, since we will check in master
|
||||
// again.
|
||||
// todo: check tenant valid? or we don't need to do this check, since we need to check in master again.
|
||||
ProcessInstance workflowInstance = request.getWorkflowInstance(); |
||||
if (workflowInstance.getState() == null || !workflowInstance.getState().isFinished()) { |
||||
throw new ExecuteRuntimeException( |
||||
String.format("The workflow instance: %s status is %s, cannot repeat running", |
||||
workflowInstance.getName(), workflowInstance.getState())); |
||||
} |
||||
Command command = Command.builder() |
||||
.commandType(CommandType.REPEAT_RUNNING) |
||||
.commandParam(JSONUtils.toJsonString(createCommandParams(workflowInstance))) |
||||
.processDefinitionCode(workflowInstance.getProcessDefinitionCode()) |
||||
.processDefinitionVersion(workflowInstance.getProcessDefinitionVersion()) |
||||
.processInstanceId(workflowInstance.getId()) |
||||
.processInstancePriority(workflowInstance.getProcessInstancePriority()) |
||||
.testFlag(workflowInstance.getTestFlag()) |
||||
.build(); |
||||
if (commandService.createCommand(command) <= 0) { |
||||
throw new ExecuteRuntimeException( |
||||
String.format("Repeat running workflow instance: %s failed, due to insert command to db failed", |
||||
workflowInstance.getName())); |
||||
} |
||||
return new RepeatRunningResult(command.getId()); |
||||
} |
||||
|
||||
@Override |
||||
public ExecuteType getExecuteType() { |
||||
return RepeatRunningExecuteFunctionBuilder.EXECUTE_TYPE; |
||||
} |
||||
|
||||
private Map<String, Object> createCommandParams(ProcessInstance workflowInstance) { |
||||
Map<String, Object> commandMap = |
||||
JSONUtils.parseObject(workflowInstance.getCommandParam(), new TypeReference<Map<String, Object>>() { |
||||
}); |
||||
Map<String, Object> repeatRunningCommandParams = new HashMap<>(); |
||||
Optional.ofNullable(MapUtils.getObject(commandMap, CMD_PARAM_START_PARAMS)) |
||||
.ifPresent(startParams -> repeatRunningCommandParams.put(CMD_PARAM_START_PARAMS, startParams)); |
||||
repeatRunningCommandParams.put(CMD_PARAM_RECOVER_PROCESS_ID_STRING, workflowInstance.getId()); |
||||
return repeatRunningCommandParams; |
||||
} |
||||
} |
@ -1,59 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.rerun; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteContext; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunction; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunctionBuilder; |
||||
import org.apache.dolphinscheduler.service.command.CommandService; |
||||
|
||||
import java.util.concurrent.CompletableFuture; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class RepeatRunningExecuteFunctionBuilder |
||||
implements |
||||
ExecuteFunctionBuilder<RepeatRunningRequest, RepeatRunningResult> { |
||||
|
||||
public static final ExecuteType EXECUTE_TYPE = ExecuteType.REPEAT_RUNNING; |
||||
|
||||
@Autowired |
||||
private CommandService commandService; |
||||
|
||||
@Override |
||||
public CompletableFuture<ExecuteFunction<RepeatRunningRequest, RepeatRunningResult>> createWorkflowInstanceExecuteFunction(ExecuteContext executeContext) { |
||||
return CompletableFuture.completedFuture(new RepeatRunningExecuteFunction(commandService)); |
||||
} |
||||
|
||||
@Override |
||||
public CompletableFuture<RepeatRunningRequest> createWorkflowInstanceExecuteRequest(ExecuteContext executeContext) { |
||||
return CompletableFuture.completedFuture( |
||||
new RepeatRunningRequest( |
||||
executeContext.getWorkflowInstance(), |
||||
executeContext.getWorkflowDefinition(), |
||||
executeContext.getExecuteUser())); |
||||
} |
||||
|
||||
@Override |
||||
public ExecuteType getExecuteType() { |
||||
return EXECUTE_TYPE; |
||||
} |
||||
} |
@ -1,83 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.stop; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunction; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteRuntimeException; |
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; |
||||
import org.apache.dolphinscheduler.extract.base.client.SingletonJdkDynamicRpcClientProxyFactory; |
||||
import org.apache.dolphinscheduler.extract.master.ITaskInstanceExecutionEventListener; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.WorkflowInstanceStateChangeEvent; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
@Slf4j |
||||
public class StopExecuteFunction implements ExecuteFunction<StopRequest, StopResult> { |
||||
|
||||
private final ProcessInstanceDao processInstanceDao; |
||||
|
||||
public StopExecuteFunction(ProcessInstanceDao processInstanceDao) { |
||||
this.processInstanceDao = processInstanceDao; |
||||
} |
||||
|
||||
@Override |
||||
public StopResult execute(StopRequest request) throws ExecuteRuntimeException { |
||||
ProcessInstance workflowInstance = request.getWorkflowInstance(); |
||||
|
||||
if (!workflowInstance.getState().canStop() |
||||
|| workflowInstance.getState() == WorkflowExecutionStatus.READY_STOP) { |
||||
throw new ExecuteRuntimeException( |
||||
String.format("The workflow instance: %s status is %s, can not be stopped", |
||||
workflowInstance.getName(), workflowInstance.getState())); |
||||
} |
||||
// update the workflow instance's status to stop
|
||||
workflowInstance.setCommandType(CommandType.STOP); |
||||
workflowInstance.addHistoryCmd(CommandType.STOP); |
||||
workflowInstance.setStateWithDesc(WorkflowExecutionStatus.READY_STOP, CommandType.STOP.getDescp() + " by user"); |
||||
if (processInstanceDao.updateById(workflowInstance)) { |
||||
log.info("Workflow instance {} ready to stop success, will call master to stop the workflow instance", |
||||
workflowInstance.getName()); |
||||
try { |
||||
// todo: direct call the workflow instance stop method
|
||||
ITaskInstanceExecutionEventListener iTaskInstanceExecutionEventListener = |
||||
SingletonJdkDynamicRpcClientProxyFactory.getProxyClient(workflowInstance.getHost(), |
||||
ITaskInstanceExecutionEventListener.class); |
||||
iTaskInstanceExecutionEventListener.onWorkflowInstanceInstanceStateChange( |
||||
new WorkflowInstanceStateChangeEvent( |
||||
workflowInstance.getId(), 0, workflowInstance.getState(), workflowInstance.getId(), 0)); |
||||
} catch (Exception e) { |
||||
throw new ExecuteRuntimeException( |
||||
String.format("WorkflowInstance: %s stop failed", workflowInstance.getName()), e); |
||||
} |
||||
// todo: use async and inject the completeFuture in the result.
|
||||
return new StopResult(workflowInstance); |
||||
} |
||||
throw new ExecuteRuntimeException( |
||||
"Workflow instance stop failed, due to update the workflow instance status failed"); |
||||
} |
||||
|
||||
@Override |
||||
public ExecuteType getExecuteType() { |
||||
return StopExecuteFunctionBuilder.EXECUTE_TYPE; |
||||
} |
||||
|
||||
} |
@ -1,53 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.stop; |
||||
|
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteContext; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunction; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteFunctionBuilder; |
||||
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; |
||||
|
||||
import java.util.concurrent.CompletableFuture; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class StopExecuteFunctionBuilder implements ExecuteFunctionBuilder<StopRequest, StopResult> { |
||||
|
||||
public static final ExecuteType EXECUTE_TYPE = ExecuteType.STOP; |
||||
|
||||
@Autowired |
||||
private ProcessInstanceDao processInstanceDao; |
||||
|
||||
@Override |
||||
public CompletableFuture<ExecuteFunction<StopRequest, StopResult>> createWorkflowInstanceExecuteFunction(ExecuteContext executeContext) { |
||||
return CompletableFuture.completedFuture(new StopExecuteFunction(processInstanceDao)); |
||||
} |
||||
|
||||
@Override |
||||
public CompletableFuture<StopRequest> createWorkflowInstanceExecuteRequest(ExecuteContext executeContext) { |
||||
return CompletableFuture.completedFuture(new StopRequest(executeContext.getWorkflowInstance())); |
||||
} |
||||
|
||||
@Override |
||||
public ExecuteType getExecuteType() { |
||||
return EXECUTE_TYPE; |
||||
} |
||||
} |
@ -1,33 +0,0 @@
|
||||
/* |
||||
* 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.api.executor.workflow.instance.stop; |
||||
|
||||
import org.apache.dolphinscheduler.api.executor.ExecuteResult; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
|
||||
import lombok.AllArgsConstructor; |
||||
import lombok.Data; |
||||
import lombok.NonNull; |
||||
|
||||
@Data |
||||
@AllArgsConstructor |
||||
public class StopResult implements ExecuteResult { |
||||
|
||||
@NonNull |
||||
private final ProcessInstance workflowInstance; |
||||
} |
@ -0,0 +1,63 @@
|
||||
/* |
||||
* 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.api.utils; |
||||
|
||||
import org.apache.dolphinscheduler.api.dto.workflow.WorkflowBackFillRequest; |
||||
import org.apache.dolphinscheduler.api.exceptions.ServiceException; |
||||
import org.apache.dolphinscheduler.common.utils.JSONUtils; |
||||
|
||||
import org.apache.commons.lang3.StringUtils; |
||||
|
||||
import java.util.ArrayList; |
||||
import java.util.Arrays; |
||||
import java.util.List; |
||||
import java.util.stream.Collectors; |
||||
|
||||
public class WorkflowUtils { |
||||
|
||||
public static List<Long> parseStartNodeList(String startNodes) { |
||||
try { |
||||
if (StringUtils.isEmpty(startNodes)) { |
||||
return new ArrayList<>(); |
||||
} |
||||
return Arrays.stream(startNodes.split(",")) |
||||
.map(String::trim) |
||||
.map(Long::parseLong) |
||||
.collect(Collectors.toList()); |
||||
} catch (Exception ex) { |
||||
throw new ServiceException("Parse startNodes: " + startNodes + " error", ex); |
||||
} |
||||
} |
||||
|
||||
public static WorkflowBackFillRequest.BackfillTime parseBackfillTime(String backfillTimeJson) { |
||||
try { |
||||
if (StringUtils.isEmpty(backfillTimeJson)) { |
||||
throw new IllegalArgumentException("backfillTime is empty"); |
||||
} |
||||
WorkflowBackFillRequest.BackfillTime backfillTime = |
||||
JSONUtils.parseObject(backfillTimeJson, WorkflowBackFillRequest.BackfillTime.class); |
||||
if (backfillTime == null) { |
||||
throw new IllegalArgumentException("backfillTime is invalid"); |
||||
} |
||||
return backfillTime; |
||||
} catch (Exception ex) { |
||||
throw new ServiceException("Parse backfillTime: " + backfillTimeJson + " error", ex); |
||||
} |
||||
} |
||||
|
||||
} |
@ -0,0 +1,101 @@
|
||||
/* |
||||
* 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.api.validator.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.common.enums.ComplementDependentMode; |
||||
import org.apache.dolphinscheduler.common.enums.ExecutionOrder; |
||||
import org.apache.dolphinscheduler.common.enums.FailureStrategy; |
||||
import org.apache.dolphinscheduler.common.enums.Flag; |
||||
import org.apache.dolphinscheduler.common.enums.Priority; |
||||
import org.apache.dolphinscheduler.common.enums.RunMode; |
||||
import org.apache.dolphinscheduler.common.enums.TaskDependType; |
||||
import org.apache.dolphinscheduler.common.enums.WarningType; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
import org.apache.dolphinscheduler.plugin.task.api.model.Property; |
||||
|
||||
import java.time.ZonedDateTime; |
||||
import java.util.List; |
||||
|
||||
import lombok.AllArgsConstructor; |
||||
import lombok.Builder; |
||||
import lombok.Data; |
||||
import lombok.NoArgsConstructor; |
||||
|
||||
@Data |
||||
@Builder |
||||
@NoArgsConstructor |
||||
@AllArgsConstructor |
||||
public class BackfillWorkflowDTO { |
||||
|
||||
private User loginUser; |
||||
|
||||
private ProcessDefinition workflowDefinition; |
||||
|
||||
private List<Long> startNodes; |
||||
|
||||
private FailureStrategy failureStrategy; |
||||
|
||||
private TaskDependType taskDependType; |
||||
|
||||
private CommandType execType; |
||||
|
||||
private WarningType warningType; |
||||
|
||||
private Integer warningGroupId; |
||||
|
||||
private RunMode runMode; |
||||
|
||||
private Priority workflowInstancePriority; |
||||
|
||||
private String workerGroup; |
||||
|
||||
private String tenantCode; |
||||
|
||||
private Long environmentCode; |
||||
|
||||
private List<Property> startParamList; |
||||
|
||||
private Flag dryRun; |
||||
|
||||
private Flag testFlag; |
||||
|
||||
private Long triggerCode; |
||||
|
||||
private BackfillParamsDTO backfillParams; |
||||
|
||||
@Data |
||||
@Builder |
||||
@NoArgsConstructor |
||||
@AllArgsConstructor |
||||
public static class BackfillParamsDTO { |
||||
|
||||
private RunMode runMode; |
||||
|
||||
private List<ZonedDateTime> backfillDateList; |
||||
|
||||
private Integer expectedParallelismNumber; |
||||
|
||||
private ComplementDependentMode backfillDependentMode; |
||||
|
||||
private boolean allLevelDependent; |
||||
private ExecutionOrder executionOrder; |
||||
} |
||||
|
||||
} |
@ -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.api.validator.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.api.validator.IValidator; |
||||
|
||||
import org.apache.commons.collections4.CollectionUtils; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class BackfillWorkflowDTOValidator implements IValidator<BackfillWorkflowDTO> { |
||||
|
||||
@Override |
||||
public void validate(final BackfillWorkflowDTO backfillWorkflowDTO) { |
||||
final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams(); |
||||
if (backfillParams == null) { |
||||
throw new IllegalArgumentException("backfillParams is null"); |
||||
} |
||||
if (CollectionUtils.isEmpty(backfillParams.getBackfillDateList())) { |
||||
throw new IllegalArgumentException("backfillDateList is empty"); |
||||
} |
||||
if (backfillParams.getExpectedParallelismNumber() < 0) { |
||||
throw new IllegalArgumentException("expectedParallelismNumber should >= 0"); |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,127 @@
|
||||
/* |
||||
* 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.api.validator.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.api.dto.workflow.WorkflowBackFillRequest; |
||||
import org.apache.dolphinscheduler.api.exceptions.ServiceException; |
||||
import org.apache.dolphinscheduler.api.utils.WorkflowUtils; |
||||
import org.apache.dolphinscheduler.api.validator.ITransformer; |
||||
import org.apache.dolphinscheduler.common.utils.CodeGenerateUtils; |
||||
import org.apache.dolphinscheduler.common.utils.DateUtils; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; |
||||
import org.apache.dolphinscheduler.dao.entity.Schedule; |
||||
import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao; |
||||
import org.apache.dolphinscheduler.dao.repository.ProjectDao; |
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.PropertyUtils; |
||||
import org.apache.dolphinscheduler.service.cron.CronUtils; |
||||
import org.apache.dolphinscheduler.service.process.ProcessService; |
||||
|
||||
import org.apache.commons.lang3.StringUtils; |
||||
|
||||
import java.time.ZonedDateTime; |
||||
import java.util.Arrays; |
||||
import java.util.List; |
||||
import java.util.stream.Collectors; |
||||
|
||||
import lombok.SneakyThrows; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Component |
||||
public class BackfillWorkflowRequestTransformer implements ITransformer<WorkflowBackFillRequest, BackfillWorkflowDTO> { |
||||
|
||||
@Autowired |
||||
private ProcessService processService; |
||||
|
||||
@Autowired |
||||
private ProjectDao projectDao; |
||||
|
||||
@Autowired |
||||
private ProcessDefinitionDao processDefinitionDao; |
||||
|
||||
@Override |
||||
public BackfillWorkflowDTO transform(WorkflowBackFillRequest workflowBackFillRequest) { |
||||
|
||||
final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = |
||||
transformBackfillParamsDTO(workflowBackFillRequest); |
||||
final BackfillWorkflowDTO backfillWorkflowDTO = BackfillWorkflowDTO.builder() |
||||
.loginUser(workflowBackFillRequest.getLoginUser()) |
||||
.startNodes(WorkflowUtils.parseStartNodeList(workflowBackFillRequest.getStartNodes())) |
||||
.failureStrategy(workflowBackFillRequest.getFailureStrategy()) |
||||
.taskDependType(workflowBackFillRequest.getTaskDependType()) |
||||
.execType(workflowBackFillRequest.getExecType()) |
||||
.warningType(workflowBackFillRequest.getWarningType()) |
||||
.warningGroupId(workflowBackFillRequest.getWarningGroupId()) |
||||
.runMode(workflowBackFillRequest.getBackfillRunMode()) |
||||
.workflowInstancePriority(workflowBackFillRequest.getWorkflowInstancePriority()) |
||||
.workerGroup(workflowBackFillRequest.getWorkerGroup()) |
||||
.tenantCode(workflowBackFillRequest.getTenantCode()) |
||||
.environmentCode(workflowBackFillRequest.getEnvironmentCode()) |
||||
.startParamList( |
||||
PropertyUtils.startParamsTransformPropertyList(workflowBackFillRequest.getStartParamList())) |
||||
.dryRun(workflowBackFillRequest.getDryRun()) |
||||
.testFlag(workflowBackFillRequest.getTestFlag()) |
||||
.triggerCode(CodeGenerateUtils.genCode()) |
||||
.backfillParams(backfillParams) |
||||
.build(); |
||||
|
||||
ProcessDefinition workflowDefinition = processDefinitionDao |
||||
.queryByCode(workflowBackFillRequest.getWorkflowDefinitionCode()) |
||||
.orElseThrow(() -> new ServiceException( |
||||
"Cannot find the workflow: " + workflowBackFillRequest.getWorkflowDefinitionCode())); |
||||
|
||||
backfillWorkflowDTO.setWorkflowDefinition(workflowDefinition); |
||||
return backfillWorkflowDTO; |
||||
} |
||||
|
||||
private BackfillWorkflowDTO.BackfillParamsDTO transformBackfillParamsDTO(WorkflowBackFillRequest workflowBackFillRequest) { |
||||
final List<ZonedDateTime> backfillDateList = parseBackfillDateList(workflowBackFillRequest); |
||||
return BackfillWorkflowDTO.BackfillParamsDTO.builder() |
||||
.runMode(workflowBackFillRequest.getBackfillRunMode()) |
||||
.expectedParallelismNumber(workflowBackFillRequest.getExpectedParallelismNumber()) |
||||
.backfillDateList(backfillDateList) |
||||
.backfillDependentMode(workflowBackFillRequest.getBackfillDependentMode()) |
||||
.allLevelDependent(workflowBackFillRequest.isAllLevelDependent()) |
||||
.executionOrder(workflowBackFillRequest.getExecutionOrder()) |
||||
.build(); |
||||
} |
||||
|
||||
@SneakyThrows |
||||
private List<ZonedDateTime> parseBackfillDateList(WorkflowBackFillRequest workflowBackFillRequest) { |
||||
final WorkflowBackFillRequest.BackfillTime backfillTime = workflowBackFillRequest.getBackfillTime(); |
||||
List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionCode( |
||||
workflowBackFillRequest.getWorkflowDefinitionCode()); |
||||
|
||||
if (StringUtils.isNotEmpty(backfillTime.getComplementStartDate()) |
||||
&& StringUtils.isNotEmpty(backfillTime.getComplementEndDate())) { |
||||
// todo: why we need to filter the schedules here?
|
||||
return CronUtils.getSelfFireDateList( |
||||
DateUtils.stringToZoneDateTime(backfillTime.getComplementStartDate()), |
||||
DateUtils.stringToZoneDateTime(backfillTime.getComplementEndDate()), |
||||
schedules); |
||||
} |
||||
if (StringUtils.isNotEmpty(backfillTime.getComplementScheduleDateList())) { |
||||
return Arrays.stream(backfillTime.getComplementScheduleDateList().split(",")) |
||||
.distinct() |
||||
.map(DateUtils::stringToZoneDateTime) |
||||
.collect(Collectors.toList()); |
||||
} |
||||
throw new ServiceException("backfillTime: " + backfillTime + " is invalid"); |
||||
} |
||||
} |
@ -0,0 +1,78 @@
|
||||
/* |
||||
* 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.api.validator.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.common.enums.FailureStrategy; |
||||
import org.apache.dolphinscheduler.common.enums.Flag; |
||||
import org.apache.dolphinscheduler.common.enums.Priority; |
||||
import org.apache.dolphinscheduler.common.enums.RunMode; |
||||
import org.apache.dolphinscheduler.common.enums.TaskDependType; |
||||
import org.apache.dolphinscheduler.common.enums.WarningType; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
import org.apache.dolphinscheduler.plugin.task.api.model.Property; |
||||
|
||||
import java.util.List; |
||||
|
||||
import lombok.AllArgsConstructor; |
||||
import lombok.Builder; |
||||
import lombok.Data; |
||||
import lombok.NoArgsConstructor; |
||||
|
||||
@Data |
||||
@Builder |
||||
@NoArgsConstructor |
||||
@AllArgsConstructor |
||||
public class TriggerWorkflowDTO { |
||||
|
||||
private User loginUser; |
||||
|
||||
private ProcessDefinition workflowDefinition; |
||||
|
||||
private List<Long> startNodes; |
||||
|
||||
private FailureStrategy failureStrategy; |
||||
|
||||
private TaskDependType taskDependType; |
||||
|
||||
private CommandType execType; |
||||
|
||||
private WarningType warningType; |
||||
|
||||
private Integer warningGroupId; |
||||
|
||||
private RunMode runMode; |
||||
|
||||
private Priority workflowInstancePriority; |
||||
|
||||
private String workerGroup; |
||||
|
||||
private String tenantCode; |
||||
|
||||
private Long environmentCode; |
||||
|
||||
private List<Property> startParamList; |
||||
|
||||
private Flag dryRun; |
||||
|
||||
private Flag testFlag; |
||||
|
||||
private Long triggerCode; |
||||
|
||||
} |
@ -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. |
||||
*/ |
||||
|
||||
package org.apache.dolphinscheduler.api.validator.workflow; |
||||
|
||||
import org.apache.dolphinscheduler.api.dto.workflow.WorkflowTriggerRequest; |
||||
import org.apache.dolphinscheduler.api.exceptions.ServiceException; |
||||
import org.apache.dolphinscheduler.api.utils.WorkflowUtils; |
||||
import org.apache.dolphinscheduler.api.validator.ITransformer; |
||||
import org.apache.dolphinscheduler.common.utils.CodeGenerateUtils; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; |
||||
import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao; |
||||
import org.apache.dolphinscheduler.plugin.task.api.utils.PropertyUtils; |
||||
|
||||
import lombok.extern.slf4j.Slf4j; |
||||
|
||||
import org.springframework.beans.factory.annotation.Autowired; |
||||
import org.springframework.stereotype.Component; |
||||
|
||||
@Slf4j |
||||
@Component |
||||
public class TriggerWorkflowRequestTransformer implements ITransformer<WorkflowTriggerRequest, TriggerWorkflowDTO> { |
||||
|
||||
@Autowired |
||||
private ProcessDefinitionDao processDefinitionDao; |
||||
|
||||
@Override |
||||
public TriggerWorkflowDTO transform(WorkflowTriggerRequest workflowTriggerRequest) { |
||||
TriggerWorkflowDTO triggerWorkflowDTO = TriggerWorkflowDTO.builder() |
||||
.loginUser(workflowTriggerRequest.getLoginUser()) |
||||
.startNodes(WorkflowUtils.parseStartNodeList(workflowTriggerRequest.getStartNodes())) |
||||
.failureStrategy(workflowTriggerRequest.getFailureStrategy()) |
||||
.taskDependType(workflowTriggerRequest.getTaskDependType()) |
||||
.execType(workflowTriggerRequest.getExecType()) |
||||
.warningType(workflowTriggerRequest.getWarningType()) |
||||
.warningGroupId(workflowTriggerRequest.getWarningGroupId()) |
||||
.workflowInstancePriority(workflowTriggerRequest.getWorkflowInstancePriority()) |
||||
.workerGroup(workflowTriggerRequest.getWorkerGroup()) |
||||
.tenantCode(workflowTriggerRequest.getTenantCode()) |
||||
.environmentCode(workflowTriggerRequest.getEnvironmentCode()) |
||||
.startParamList( |
||||
PropertyUtils.startParamsTransformPropertyList(workflowTriggerRequest.getStartParamList())) |
||||
.dryRun(workflowTriggerRequest.getDryRun()) |
||||
.testFlag(workflowTriggerRequest.getTestFlag()) |
||||
.triggerCode(CodeGenerateUtils.genCode()) |
||||
.build(); |
||||
|
||||
ProcessDefinition workflowDefinition = processDefinitionDao |
||||
.queryByCode(workflowTriggerRequest.getWorkflowDefinitionCode()) |
||||
.orElseThrow(() -> new ServiceException( |
||||
"Cannot find the workflow: " + workflowTriggerRequest.getWorkflowDefinitionCode())); |
||||
|
||||
triggerWorkflowDTO.setWorkflowDefinition(workflowDefinition); |
||||
return triggerWorkflowDTO; |
||||
} |
||||
} |
@ -0,0 +1,105 @@
|
||||
/* |
||||
* 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.api.executor.workflow; |
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; |
||||
import static org.junit.jupiter.api.Assertions.assertThrows; |
||||
|
||||
import org.apache.dolphinscheduler.api.exceptions.ServiceException; |
||||
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; |
||||
|
||||
import org.junit.jupiter.api.Assertions; |
||||
import org.junit.jupiter.api.extension.ExtendWith; |
||||
import org.junit.jupiter.params.ParameterizedTest; |
||||
import org.junit.jupiter.params.provider.EnumSource; |
||||
import org.mockito.InjectMocks; |
||||
import org.mockito.Mock; |
||||
import org.mockito.junit.jupiter.MockitoExtension; |
||||
import org.mockito.junit.jupiter.MockitoSettings; |
||||
import org.mockito.quality.Strictness; |
||||
|
||||
@ExtendWith(MockitoExtension.class) |
||||
@MockitoSettings(strictness = Strictness.LENIENT) |
||||
class StopWorkflowInstanceExecuteFunctionTest { |
||||
|
||||
@Mock |
||||
private ProcessInstanceDao processInstanceDao; |
||||
|
||||
@InjectMocks |
||||
private StopWorkflowInstanceExecutorDelegate stopWorkflowInstanceExecutorDelegate; |
||||
|
||||
@ParameterizedTest |
||||
@EnumSource(value = WorkflowExecutionStatus.class, names = { |
||||
"RUNNING_EXECUTION", |
||||
"READY_PAUSE", |
||||
"READY_STOP", |
||||
"SERIAL_WAIT", |
||||
"WAIT_TO_RUN"}) |
||||
void exceptionIfWorkflowInstanceCannotStop_canStop(WorkflowExecutionStatus workflowExecutionStatus) { |
||||
ProcessInstance workflowInstance = new ProcessInstance(); |
||||
workflowInstance.setName("Workflow-1"); |
||||
workflowInstance.setState(workflowExecutionStatus); |
||||
assertDoesNotThrow( |
||||
() -> stopWorkflowInstanceExecutorDelegate.exceptionIfWorkflowInstanceCannotStop(workflowInstance)); |
||||
} |
||||
|
||||
@ParameterizedTest |
||||
@EnumSource(value = WorkflowExecutionStatus.class, names = { |
||||
"RUNNING_EXECUTION", |
||||
"READY_PAUSE", |
||||
"READY_STOP", |
||||
"SERIAL_WAIT", |
||||
"WAIT_TO_RUN"}, mode = EnumSource.Mode.EXCLUDE) |
||||
void exceptionIfWorkflowInstanceCannotStop_canNotStop(WorkflowExecutionStatus workflowExecutionStatus) { |
||||
ProcessInstance workflowInstance = new ProcessInstance(); |
||||
workflowInstance.setName("Workflow-1"); |
||||
workflowInstance.setState(workflowExecutionStatus); |
||||
ServiceException serviceException = assertThrows(ServiceException.class, |
||||
() -> stopWorkflowInstanceExecutorDelegate.exceptionIfWorkflowInstanceCannotStop(workflowInstance)); |
||||
Assertions.assertEquals( |
||||
"Internal Server Error: The workflow instance: Workflow-1 status is " + workflowExecutionStatus |
||||
+ ", can not stop", |
||||
serviceException.getMessage()); |
||||
} |
||||
|
||||
@ParameterizedTest |
||||
@EnumSource(value = WorkflowExecutionStatus.class, names = { |
||||
"SERIAL_WAIT", |
||||
"WAIT_TO_RUN"}) |
||||
void ifWorkflowInstanceCanDirectStopInDB_canDirectStopInDB(WorkflowExecutionStatus workflowExecutionStatus) { |
||||
ProcessInstance workflowInstance = new ProcessInstance(); |
||||
workflowInstance.setName("Workflow-1"); |
||||
workflowInstance.setState(workflowExecutionStatus); |
||||
Assertions |
||||
.assertTrue(stopWorkflowInstanceExecutorDelegate.ifWorkflowInstanceCanDirectStopInDB(workflowInstance)); |
||||
} |
||||
|
||||
@ParameterizedTest |
||||
@EnumSource(value = WorkflowExecutionStatus.class, names = { |
||||
"SERIAL_WAIT", |
||||
"WAIT_TO_RUN"}, mode = EnumSource.Mode.EXCLUDE) |
||||
void ifWorkflowInstanceCanDirectStopInDB_canNotDirectStopInDB(WorkflowExecutionStatus workflowExecutionStatus) { |
||||
ProcessInstance workflowInstance = new ProcessInstance(); |
||||
workflowInstance.setName("Workflow-1"); |
||||
workflowInstance.setState(workflowExecutionStatus); |
||||
Assertions.assertFalse( |
||||
stopWorkflowInstanceExecutorDelegate.ifWorkflowInstanceCanDirectStopInDB(workflowInstance)); |
||||
} |
||||
} |
@ -1,703 +0,0 @@
|
||||
/* |
||||
* 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.api.service; |
||||
|
||||
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.RERUN; |
||||
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_START; |
||||
import static org.mockito.ArgumentMatchers.any; |
||||
import static org.mockito.ArgumentMatchers.argThat; |
||||
import static org.mockito.Mockito.RETURNS_DEEP_STUBS; |
||||
import static org.mockito.Mockito.doReturn; |
||||
import static org.mockito.Mockito.times; |
||||
import static org.mockito.Mockito.verify; |
||||
import static org.mockito.Mockito.when; |
||||
|
||||
import org.apache.dolphinscheduler.api.dto.workflowInstance.WorkflowExecuteResponse; |
||||
import org.apache.dolphinscheduler.api.enums.ExecuteType; |
||||
import org.apache.dolphinscheduler.api.enums.Status; |
||||
import org.apache.dolphinscheduler.api.exceptions.ServiceException; |
||||
import org.apache.dolphinscheduler.api.executor.ExecuteClient; |
||||
import org.apache.dolphinscheduler.api.permission.ResourcePermissionCheckService; |
||||
import org.apache.dolphinscheduler.api.service.impl.ExecutorServiceImpl; |
||||
import org.apache.dolphinscheduler.api.service.impl.ProjectServiceImpl; |
||||
import org.apache.dolphinscheduler.common.constants.Constants; |
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.common.enums.ComplementDependentMode; |
||||
import org.apache.dolphinscheduler.common.enums.ExecutionOrder; |
||||
import org.apache.dolphinscheduler.common.enums.FailureStrategy; |
||||
import org.apache.dolphinscheduler.common.enums.Priority; |
||||
import org.apache.dolphinscheduler.common.enums.ReleaseState; |
||||
import org.apache.dolphinscheduler.common.enums.RunMode; |
||||
import org.apache.dolphinscheduler.common.enums.TaskDependType; |
||||
import org.apache.dolphinscheduler.common.enums.TaskGroupQueueStatus; |
||||
import org.apache.dolphinscheduler.common.enums.WarningType; |
||||
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus; |
||||
import org.apache.dolphinscheduler.common.model.Server; |
||||
import org.apache.dolphinscheduler.dao.entity.Command; |
||||
import org.apache.dolphinscheduler.dao.entity.DependentProcessDefinition; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessDefinition; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessInstance; |
||||
import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation; |
||||
import org.apache.dolphinscheduler.dao.entity.Project; |
||||
import org.apache.dolphinscheduler.dao.entity.Schedule; |
||||
import org.apache.dolphinscheduler.dao.entity.TaskGroupQueue; |
||||
import org.apache.dolphinscheduler.dao.entity.Tenant; |
||||
import org.apache.dolphinscheduler.dao.entity.User; |
||||
import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper; |
||||
import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper; |
||||
import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper; |
||||
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper; |
||||
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper; |
||||
import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper; |
||||
import org.apache.dolphinscheduler.dao.mapper.TaskGroupQueueMapper; |
||||
import org.apache.dolphinscheduler.dao.mapper.TenantMapper; |
||||
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao; |
||||
import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils; |
||||
import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType; |
||||
import org.apache.dolphinscheduler.service.command.CommandService; |
||||
import org.apache.dolphinscheduler.service.process.ProcessService; |
||||
import org.apache.dolphinscheduler.service.process.TriggerRelationService; |
||||
|
||||
import java.util.ArrayList; |
||||
import java.util.Collections; |
||||
import java.util.Date; |
||||
import java.util.HashMap; |
||||
import java.util.LinkedList; |
||||
import java.util.List; |
||||
import java.util.Map; |
||||
import java.util.Optional; |
||||
|
||||
import org.assertj.core.util.Lists; |
||||
import org.junit.jupiter.api.Assertions; |
||||
import org.junit.jupiter.api.BeforeEach; |
||||
import org.junit.jupiter.api.Test; |
||||
import org.junit.jupiter.api.extension.ExtendWith; |
||||
import org.mockito.InjectMocks; |
||||
import org.mockito.Mock; |
||||
import org.mockito.Mockito; |
||||
import org.mockito.junit.jupiter.MockitoExtension; |
||||
import org.mockito.junit.jupiter.MockitoSettings; |
||||
import org.mockito.quality.Strictness; |
||||
import org.slf4j.Logger; |
||||
import org.slf4j.LoggerFactory; |
||||
|
||||
/** |
||||
* executor service 2 test |
||||
*/ |
||||
@ExtendWith(MockitoExtension.class) |
||||
@MockitoSettings(strictness = Strictness.LENIENT) |
||||
public class ExecuteFunctionServiceTest { |
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(ExecuteFunctionServiceTest.class); |
||||
|
||||
@Mock |
||||
private ResourcePermissionCheckService resourcePermissionCheckService; |
||||
|
||||
@InjectMocks |
||||
private ExecutorServiceImpl executorService; |
||||
|
||||
@Mock |
||||
private ProcessService processService; |
||||
|
||||
@Mock |
||||
private CommandService commandService; |
||||
|
||||
@Mock |
||||
private WorkerGroupService workerGroupService; |
||||
|
||||
@Mock |
||||
private ProcessDefinitionMapper processDefinitionMapper; |
||||
|
||||
@Mock |
||||
private ProcessTaskRelationMapper processTaskRelationMapper; |
||||
|
||||
@Mock |
||||
private TaskDefinitionMapper taskDefinitionMapper; |
||||
|
||||
@Mock |
||||
private TaskDefinitionLogMapper taskDefinitionLogMapper; |
||||
|
||||
@Mock |
||||
private ProjectMapper projectMapper; |
||||
|
||||
@Mock |
||||
private TenantMapper tenantMapper; |
||||
|
||||
@Mock |
||||
private ProjectServiceImpl projectService; |
||||
|
||||
@Mock |
||||
private MonitorService monitorService; |
||||
|
||||
@Mock |
||||
private TaskGroupQueueMapper taskGroupQueueMapper; |
||||
|
||||
@Mock |
||||
private ProcessInstanceMapper processInstanceMapper; |
||||
|
||||
@Mock |
||||
private TriggerRelationService triggerRelationService; |
||||
|
||||
@Mock |
||||
private ExecuteClient executeClient; |
||||
|
||||
@Mock |
||||
private ProcessInstanceDao processInstanceDao; |
||||
|
||||
@Mock |
||||
private ProcessDefinitionService processDefinitionService; |
||||
|
||||
@Mock |
||||
private ProcessLineageService processLineageService; |
||||
|
||||
private int processDefinitionId = 1; |
||||
|
||||
private int processDefinitionVersion = 1; |
||||
|
||||
private long processDefinitionCode = 1L; |
||||
|
||||
private int processInstanceId = 1; |
||||
|
||||
private String tenantCode = "root"; |
||||
|
||||
private int userId = 1; |
||||
|
||||
private int taskQueueId = 1; |
||||
|
||||
private ProcessDefinition processDefinition = new ProcessDefinition(); |
||||
|
||||
private ProcessInstance processInstance = new ProcessInstance(); |
||||
|
||||
private TaskGroupQueue taskGroupQueue = new TaskGroupQueue(); |
||||
|
||||
private List<ProcessTaskRelation> processTaskRelations = new ArrayList<>(); |
||||
|
||||
private User loginUser = new User(); |
||||
|
||||
private long projectCode = 1L; |
||||
|
||||
private String projectName = "projectName"; |
||||
|
||||
private Project project = new Project(); |
||||
|
||||
private String cronTime; |
||||
|
||||
@BeforeEach |
||||
public void init() { |
||||
// user
|
||||
loginUser.setId(userId); |
||||
|
||||
// processDefinition
|
||||
processDefinition.setId(processDefinitionId); |
||||
processDefinition.setReleaseState(ReleaseState.ONLINE); |
||||
processDefinition.setUserId(userId); |
||||
processDefinition.setVersion(1); |
||||
processDefinition.setCode(1L); |
||||
processDefinition.setProjectCode(projectCode); |
||||
|
||||
// processInstance
|
||||
processInstance.setId(processInstanceId); |
||||
processInstance.setProjectCode(projectCode); |
||||
processInstance.setState(WorkflowExecutionStatus.FAILURE); |
||||
processInstance.setExecutorId(userId); |
||||
processInstance.setHost("127.0.0.1:5678"); |
||||
processInstance.setProcessDefinitionVersion(1); |
||||
processInstance.setProcessDefinitionCode(1L); |
||||
|
||||
// project
|
||||
project.setCode(projectCode); |
||||
project.setName(projectName); |
||||
|
||||
// taskGroupQueue
|
||||
taskGroupQueue.setId(taskQueueId); |
||||
taskGroupQueue.setStatus(TaskGroupQueueStatus.WAIT_QUEUE); |
||||
taskGroupQueue.setProcessId(processInstanceId); |
||||
|
||||
// cronRangeTime
|
||||
cronTime = "2020-01-01 00:00:00,2020-01-31 23:00:00"; |
||||
|
||||
// processTaskRelations
|
||||
ProcessTaskRelation processTaskRelation1 = new ProcessTaskRelation(); |
||||
processTaskRelation1.setPostTaskCode(123456789L); |
||||
ProcessTaskRelation processTaskRelation2 = new ProcessTaskRelation(); |
||||
processTaskRelation2.setPostTaskCode(987654321L); |
||||
processTaskRelations.add(processTaskRelation1); |
||||
processTaskRelations.add(processTaskRelation2); |
||||
|
||||
// mock
|
||||
Mockito.when(projectMapper.queryByCode(projectCode)).thenReturn(project); |
||||
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_START)) |
||||
.thenReturn(checkProjectAndAuth()); |
||||
Mockito.when(processDefinitionMapper.queryByCode(processDefinitionCode)).thenReturn(this.processDefinition); |
||||
Mockito.when(processService.getTenantForProcess(tenantCode, userId)).thenReturn(tenantCode); |
||||
doReturn(1).when(commandService).createCommand(argThat(c -> c.getId() == null)); |
||||
doReturn(0).when(commandService).createCommand(argThat(c -> c.getId() != null)); |
||||
Mockito.when(monitorService.listServer(RegistryNodeType.MASTER)).thenReturn(getMasterServersList()); |
||||
Mockito.when(processService.findProcessInstanceDetailById(processInstanceId)) |
||||
.thenReturn(Optional.ofNullable(processInstance)); |
||||
Mockito.when(processService.findProcessDefinition(1L, 1)).thenReturn(this.processDefinition); |
||||
Mockito.when(taskGroupQueueMapper.selectById(1)).thenReturn(taskGroupQueue); |
||||
Mockito.when(processInstanceMapper.selectById(1)).thenReturn(processInstance); |
||||
Mockito.when(triggerRelationService.saveProcessInstanceTrigger(Mockito.any(), Mockito.any())) |
||||
.thenReturn(1); |
||||
Mockito.when(processService.findRelationByCode(processDefinitionCode, processDefinitionVersion)) |
||||
.thenReturn(processTaskRelations); |
||||
} |
||||
|
||||
/** |
||||
* not complement |
||||
*/ |
||||
@Test |
||||
public void testNoComplement() { |
||||
|
||||
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)) |
||||
.thenReturn(zeroSchedulerList()); |
||||
Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant()); |
||||
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode, |
||||
processDefinitionCode, |
||||
"{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", |
||||
CommandType.START_PROCESS, |
||||
null, null, |
||||
null, null, null, |
||||
RunMode.RUN_MODE_SERIAL, |
||||
Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 10, null, null, |
||||
Constants.DRY_RUN_FLAG_NO, |
||||
Constants.TEST_FLAG_NO, |
||||
ComplementDependentMode.OFF_MODE, null, |
||||
false, |
||||
ExecutionOrder.DESC_ORDER); |
||||
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); |
||||
verify(commandService, times(1)).createCommand(any(Command.class)); |
||||
|
||||
} |
||||
|
||||
/** |
||||
* not complement |
||||
*/ |
||||
@Test |
||||
public void testComplementWithStartNodeList() { |
||||
|
||||
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)) |
||||
.thenReturn(zeroSchedulerList()); |
||||
Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant()); |
||||
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode, |
||||
processDefinitionCode, |
||||
"{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", |
||||
CommandType.START_PROCESS, |
||||
null, "123456789,987654321", |
||||
null, null, null, |
||||
RunMode.RUN_MODE_SERIAL, |
||||
Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, null, |
||||
Constants.DRY_RUN_FLAG_NO, |
||||
Constants.TEST_FLAG_NO, |
||||
ComplementDependentMode.OFF_MODE, null, |
||||
false, |
||||
ExecutionOrder.DESC_ORDER); |
||||
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); |
||||
verify(commandService, times(1)).createCommand(any(Command.class)); |
||||
|
||||
} |
||||
|
||||
@Test |
||||
public void testComplementWithOldStartNodeList() { |
||||
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)) |
||||
.thenReturn(zeroSchedulerList()); |
||||
Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant()); |
||||
Map<String, Object> result = new HashMap<>(); |
||||
try { |
||||
result = executorService.execProcessInstance(loginUser, projectCode, |
||||
processDefinitionCode, |
||||
"{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", |
||||
CommandType.START_PROCESS, |
||||
null, "1123456789,987654321", |
||||
null, null, null, |
||||
RunMode.RUN_MODE_SERIAL, |
||||
Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, 0, |
||||
Constants.DRY_RUN_FLAG_NO, |
||||
Constants.TEST_FLAG_NO, |
||||
ComplementDependentMode.OFF_MODE, null, |
||||
false, |
||||
ExecutionOrder.DESC_ORDER); |
||||
} catch (ServiceException e) { |
||||
Assertions.assertEquals(Status.START_NODE_NOT_EXIST_IN_LAST_PROCESS.getCode(), e.getCode()); |
||||
} |
||||
} |
||||
|
||||
@Test |
||||
public void testComplementWithDependentMode() { |
||||
Schedule schedule = new Schedule(); |
||||
schedule.setStartTime(new Date()); |
||||
schedule.setEndTime(new Date()); |
||||
schedule.setCrontab("0 0 7 * * ? *"); |
||||
schedule.setFailureStrategy(FailureStrategy.CONTINUE); |
||||
schedule.setReleaseState(ReleaseState.OFFLINE); |
||||
schedule.setWarningType(WarningType.NONE); |
||||
schedule.setCreateTime(new Date()); |
||||
schedule.setUpdateTime(new Date()); |
||||
List<Schedule> schedules = Lists.newArrayList(schedule); |
||||
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode( |
||||
processDefinitionCode)) |
||||
.thenReturn(schedules); |
||||
|
||||
DependentProcessDefinition dependentProcessDefinition = new DependentProcessDefinition(); |
||||
dependentProcessDefinition.setProcessDefinitionCode(2); |
||||
dependentProcessDefinition.setProcessDefinitionVersion(1); |
||||
dependentProcessDefinition.setTaskDefinitionCode(1); |
||||
dependentProcessDefinition.setWorkerGroup(WorkerGroupUtils.getDefaultWorkerGroup()); |
||||
dependentProcessDefinition.setTaskParams( |
||||
"{\"localParams\":[],\"resourceList\":[],\"dependence\":{\"relation\":\"AND\",\"dependTaskList\":[{\"relation\":\"AND\",\"dependItemList\":[{\"depTaskCode\":2,\"status\":\"SUCCESS\"}]}]},\"conditionResult\":{\"successNode\":[1],\"failedNode\":[1]}}"); |
||||
Mockito.when(processLineageService.queryDownstreamDependentProcessDefinitions(processDefinitionCode)) |
||||
.thenReturn(Lists.newArrayList(dependentProcessDefinition)); |
||||
|
||||
Map<Long, String> processDefinitionWorkerGroupMap = new HashMap<>(); |
||||
processDefinitionWorkerGroupMap.put(1L, WorkerGroupUtils.getDefaultWorkerGroup()); |
||||
Mockito.when(workerGroupService.queryWorkerGroupByProcessDefinitionCodes(Lists.newArrayList(1L))) |
||||
.thenReturn(processDefinitionWorkerGroupMap); |
||||
|
||||
Command command = new Command(); |
||||
command.setId(1); |
||||
command.setCommandType(CommandType.COMPLEMENT_DATA); |
||||
command.setCommandParam( |
||||
"{\"StartNodeList\":\"1\",\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}"); |
||||
command.setWorkerGroup(WorkerGroupUtils.getDefaultWorkerGroup()); |
||||
command.setProcessDefinitionCode(processDefinitionCode); |
||||
command.setExecutorId(1); |
||||
|
||||
// not enable allLevelDependent
|
||||
int count = executorService.createComplementDependentCommand(schedules, command, false); |
||||
Assertions.assertEquals(1, count); |
||||
|
||||
// enable allLevelDependent
|
||||
DependentProcessDefinition childDependent = new DependentProcessDefinition(); |
||||
childDependent.setProcessDefinitionCode(3); |
||||
childDependent.setProcessDefinitionVersion(1); |
||||
childDependent.setTaskDefinitionCode(4); |
||||
childDependent.setWorkerGroup(WorkerGroupUtils.getDefaultWorkerGroup()); |
||||
childDependent.setTaskParams( |
||||
"{\"localParams\":[],\"resourceList\":[],\"dependence\":{\"relation\":\"AND\",\"dependTaskList\":[{\"relation\":\"AND\",\"dependItemList\":[{\"depTaskCode\":3,\"status\":\"SUCCESS\"}]}]},\"conditionResult\":{\"successNode\":[1],\"failedNode\":[1]}}"); |
||||
Mockito.when(processLineageService.queryDownstreamDependentProcessDefinitions( |
||||
dependentProcessDefinition.getProcessDefinitionCode())).thenReturn(Lists.newArrayList(childDependent)) |
||||
.thenReturn(Lists.newArrayList()); |
||||
int allLevelDependentCount = executorService.createComplementDependentCommand(schedules, command, true); |
||||
Assertions.assertEquals(2, allLevelDependentCount); |
||||
} |
||||
|
||||
/** |
||||
* date error |
||||
*/ |
||||
@Test |
||||
public void testDateError() { |
||||
|
||||
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)) |
||||
.thenReturn(zeroSchedulerList()); |
||||
Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant()); |
||||
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode, |
||||
processDefinitionCode, |
||||
"{\"complementStartDate\":\"2022-01-07 12:12:12\",\"complementEndDate\":\"2022-01-06 12:12:12\"}", |
||||
CommandType.COMPLEMENT_DATA, |
||||
null, null, |
||||
null, null, null, |
||||
RunMode.RUN_MODE_SERIAL, |
||||
Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, 2, |
||||
Constants.DRY_RUN_FLAG_NO, |
||||
Constants.TEST_FLAG_NO, |
||||
ComplementDependentMode.OFF_MODE, null, |
||||
false, |
||||
ExecutionOrder.DESC_ORDER); |
||||
Assertions.assertEquals(Status.START_PROCESS_INSTANCE_ERROR, result.get(Constants.STATUS)); |
||||
verify(commandService, times(0)).createCommand(any(Command.class)); |
||||
} |
||||
|
||||
/** |
||||
* serial |
||||
*/ |
||||
@Test |
||||
public void testSerial() { |
||||
|
||||
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)) |
||||
.thenReturn(zeroSchedulerList()); |
||||
Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant()); |
||||
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode, |
||||
processDefinitionCode, |
||||
"{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", |
||||
CommandType.COMPLEMENT_DATA, |
||||
null, null, |
||||
null, null, null, |
||||
RunMode.RUN_MODE_SERIAL, |
||||
Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, null, |
||||
Constants.DRY_RUN_FLAG_NO, |
||||
Constants.TEST_FLAG_NO, |
||||
ComplementDependentMode.OFF_MODE, null, |
||||
false, |
||||
ExecutionOrder.DESC_ORDER); |
||||
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); |
||||
verify(commandService, times(1)).createCommand(any(Command.class)); |
||||
} |
||||
|
||||
/** |
||||
* without schedule |
||||
*/ |
||||
@Test |
||||
public void testParallelWithOutSchedule() { |
||||
|
||||
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)) |
||||
.thenReturn(zeroSchedulerList()); |
||||
Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant()); |
||||
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode, |
||||
processDefinitionCode, |
||||
"{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", |
||||
CommandType.COMPLEMENT_DATA, |
||||
null, null, |
||||
null, null, null, |
||||
RunMode.RUN_MODE_PARALLEL, |
||||
Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, 2, |
||||
Constants.DRY_RUN_FLAG_NO, |
||||
Constants.TEST_FLAG_NO, |
||||
ComplementDependentMode.OFF_MODE, null, |
||||
false, |
||||
ExecutionOrder.DESC_ORDER); |
||||
|
||||
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); |
||||
verify(commandService, times(2)).createCommand(any(Command.class)); |
||||
} |
||||
|
||||
/** |
||||
* with schedule |
||||
*/ |
||||
@Test |
||||
public void testParallelWithSchedule() { |
||||
|
||||
Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode)) |
||||
.thenReturn(oneSchedulerList()); |
||||
Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant()); |
||||
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode, |
||||
processDefinitionCode, |
||||
"{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", |
||||
CommandType.COMPLEMENT_DATA, |
||||
null, null, |
||||
null, null, null, |
||||
RunMode.RUN_MODE_PARALLEL, |
||||
Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, 15, |
||||
Constants.DRY_RUN_FLAG_NO, |
||||
Constants.TEST_FLAG_NO, |
||||
ComplementDependentMode.OFF_MODE, null, |
||||
false, |
||||
ExecutionOrder.DESC_ORDER); |
||||
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); |
||||
verify(commandService, times(15)).createCommand(any(Command.class)); |
||||
|
||||
} |
||||
|
||||
@Test |
||||
public void testNoMasterServers() { |
||||
Mockito.when(monitorService.listServer(RegistryNodeType.MASTER)).thenReturn(new ArrayList<>()); |
||||
|
||||
Assertions.assertThrows(ServiceException.class, () -> executorService.execProcessInstance( |
||||
loginUser, |
||||
projectCode, |
||||
processDefinitionCode, |
||||
"{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", |
||||
CommandType.COMPLEMENT_DATA, |
||||
null, |
||||
null, |
||||
null, |
||||
null, |
||||
null, |
||||
RunMode.RUN_MODE_PARALLEL, |
||||
Priority.LOW, |
||||
WorkerGroupUtils.getDefaultWorkerGroup(), |
||||
tenantCode, |
||||
100L, |
||||
110, |
||||
null, |
||||
null, |
||||
Constants.DRY_RUN_FLAG_NO, |
||||
Constants.TEST_FLAG_NO, |
||||
ComplementDependentMode.OFF_MODE, null, |
||||
false, |
||||
ExecutionOrder.DESC_ORDER)); |
||||
} |
||||
|
||||
@Test |
||||
public void testExecuteRepeatRunning() { |
||||
when(commandService.verifyIsNeedCreateCommand(any(Command.class))).thenReturn(true); |
||||
when(projectService.checkProjectAndAuth(loginUser, project, projectCode, RERUN)) |
||||
.thenReturn(checkProjectAndAuth()); |
||||
when(processInstanceDao.queryOptionalById(processInstanceId)).thenReturn(Optional.of(processInstance)); |
||||
when(processDefinitionService.queryWorkflowDefinitionThrowExceptionIfNotFound(processDefinitionCode, |
||||
processDefinitionVersion)).thenReturn(processDefinition); |
||||
Map<String, Object> result = |
||||
executorService.execute(loginUser, projectCode, processInstanceId, ExecuteType.REPEAT_RUNNING); |
||||
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); |
||||
} |
||||
|
||||
@Test |
||||
public void testOfTestRun() { |
||||
Mockito.when(commandService.verifyIsNeedCreateCommand(any(Command.class))).thenReturn(true); |
||||
Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectCode, RERUN)) |
||||
.thenReturn(checkProjectAndAuth()); |
||||
Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant()); |
||||
Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode, |
||||
processDefinitionCode, |
||||
"{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}", |
||||
CommandType.COMPLEMENT_DATA, |
||||
null, null, |
||||
null, null, 0, |
||||
RunMode.RUN_MODE_PARALLEL, |
||||
Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, 15, |
||||
Constants.DRY_RUN_FLAG_NO, |
||||
Constants.TEST_FLAG_YES, |
||||
ComplementDependentMode.OFF_MODE, null, |
||||
false, |
||||
ExecutionOrder.DESC_ORDER); |
||||
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); |
||||
} |
||||
|
||||
@Test |
||||
public void testStartCheckByProcessDefinedCode() { |
||||
List<Long> ids = Lists.newArrayList(1L); |
||||
when(processService.findAllSubWorkflowDefinitionCode(1)).thenReturn(ids); |
||||
|
||||
List<ProcessDefinition> processDefinitionList = new ArrayList<>(); |
||||
ProcessDefinition processDefinition = new ProcessDefinition(); |
||||
processDefinition.setId(1); |
||||
processDefinition.setReleaseState(ReleaseState.ONLINE); |
||||
processDefinitionList.add(processDefinition); |
||||
Mockito.when(processDefinitionMapper.queryDefinitionListByIdList(new Integer[ids.size()])) |
||||
.thenReturn(processDefinitionList); |
||||
|
||||
Map<String, Object> result = executorService.startCheckByProcessDefinedCode(1L); |
||||
Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS)); |
||||
} |
||||
|
||||
private List<Server> getMasterServersList() { |
||||
List<Server> masterServerList = new ArrayList<>(); |
||||
Server masterServer1 = new Server(); |
||||
masterServer1.setId(1); |
||||
masterServer1.setHost("192.168.220.188"); |
||||
masterServer1.setPort(1121); |
||||
masterServerList.add(masterServer1); |
||||
|
||||
Server masterServer2 = new Server(); |
||||
masterServer2.setId(2); |
||||
masterServer2.setHost("192.168.220.189"); |
||||
masterServer2.setPort(1122); |
||||
masterServerList.add(masterServer2); |
||||
|
||||
return masterServerList; |
||||
} |
||||
|
||||
private List zeroSchedulerList() { |
||||
return Collections.EMPTY_LIST; |
||||
} |
||||
|
||||
private List<Schedule> oneSchedulerList() { |
||||
List<Schedule> schedulerList = new LinkedList<>(); |
||||
Schedule schedule = new Schedule(); |
||||
schedule.setCrontab("0 0 0 1/2 * ?"); |
||||
schedulerList.add(schedule); |
||||
return schedulerList; |
||||
} |
||||
|
||||
private Map<String, Object> checkProjectAndAuth() { |
||||
Map<String, Object> result = new HashMap<>(); |
||||
result.put(Constants.STATUS, Status.SUCCESS); |
||||
return result; |
||||
} |
||||
|
||||
@Test |
||||
public void testCreateComplementToParallel() { |
||||
List<String> result = new ArrayList<>(); |
||||
int expectedParallelismNumber = 3; |
||||
LinkedList<Integer> listDate = new LinkedList<>(); |
||||
listDate.add(0); |
||||
listDate.add(1); |
||||
listDate.add(2); |
||||
listDate.add(3); |
||||
listDate.add(4); |
||||
|
||||
int listDateSize = listDate.size(); |
||||
int createCount = Math.min(listDate.size(), expectedParallelismNumber); |
||||
logger.info("In parallel mode, current expectedParallelismNumber:{}", createCount); |
||||
|
||||
int itemsPerCommand = (listDateSize / createCount); |
||||
int remainingItems = (listDateSize % createCount); |
||||
int startDateIndex = 0; |
||||
int endDateIndex = 0; |
||||
|
||||
for (int i = 1; i <= createCount; i++) { |
||||
int extra = (i <= remainingItems) ? 1 : 0; |
||||
int singleCommandItems = (itemsPerCommand + extra); |
||||
|
||||
if (i == 1) { |
||||
endDateIndex += singleCommandItems - 1; |
||||
} else { |
||||
startDateIndex = endDateIndex + 1; |
||||
endDateIndex += singleCommandItems; |
||||
} |
||||
|
||||
logger.info("startDate:{}, endDate:{}", listDate.get(startDateIndex), listDate.get(endDateIndex)); |
||||
result.add(listDate.get(startDateIndex) + "," + listDate.get(endDateIndex)); |
||||
} |
||||
|
||||
Assertions.assertEquals("0,1", result.get(0)); |
||||
Assertions.assertEquals("2,3", result.get(1)); |
||||
Assertions.assertEquals("4,4", result.get(2)); |
||||
} |
||||
|
||||
@Test |
||||
public void testExecuteTask() { |
||||
String startNodeList = "1234567870"; |
||||
TaskDependType taskDependType = TaskDependType.TASK_ONLY; |
||||
|
||||
ProcessInstance processInstanceMock = Mockito.mock(ProcessInstance.class, RETURNS_DEEP_STUBS); |
||||
Mockito.when(processService.findProcessInstanceDetailById(processInstanceId)) |
||||
.thenReturn(Optional.ofNullable(processInstanceMock)); |
||||
|
||||
ProcessDefinition processDefinition = new ProcessDefinition(); |
||||
processDefinition.setProjectCode(projectCode); |
||||
Mockito.when(processService.findProcessDefinition(Mockito.anyLong(), Mockito.anyInt())) |
||||
.thenReturn(processDefinition); |
||||
|
||||
Mockito.when(processService.getTenantForProcess(Mockito.anyString(), Mockito.anyInt())).thenReturn(tenantCode); |
||||
|
||||
when(processInstanceMock.getState().isFinished()).thenReturn(false); |
||||
WorkflowExecuteResponse responseInstanceIsNotFinished = |
||||
executorService.executeTask(loginUser, projectCode, processInstanceId, startNodeList, taskDependType); |
||||
Assertions.assertEquals(Status.WORKFLOW_INSTANCE_IS_NOT_FINISHED.getCode(), |
||||
responseInstanceIsNotFinished.getCode()); |
||||
|
||||
when(processInstanceMock.getState().isFinished()).thenReturn(true); |
||||
WorkflowExecuteResponse responseStartNodeListError = |
||||
executorService.executeTask(loginUser, projectCode, processInstanceId, "1234567870,", taskDependType); |
||||
Assertions.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), responseStartNodeListError.getCode()); |
||||
|
||||
Mockito.when(taskDefinitionLogMapper.queryMaxVersionForDefinition(Mockito.anyLong())).thenReturn(null); |
||||
WorkflowExecuteResponse responseNotDefineTask = |
||||
executorService.executeTask(loginUser, projectCode, processInstanceId, startNodeList, taskDependType); |
||||
Assertions.assertEquals(Status.EXECUTE_NOT_DEFINE_TASK.getCode(), responseNotDefineTask.getCode()); |
||||
|
||||
Mockito.when(taskDefinitionLogMapper.queryMaxVersionForDefinition(Mockito.anyLong())).thenReturn(1); |
||||
Mockito.when(commandService.verifyIsNeedCreateCommand(any())).thenReturn(true); |
||||
WorkflowExecuteResponse responseSuccess = |
||||
executorService.executeTask(loginUser, projectCode, processInstanceId, startNodeList, taskDependType); |
||||
Assertions.assertEquals(Status.SUCCESS.getCode(), responseSuccess.getCode()); |
||||
|
||||
} |
||||
|
||||
} |
@ -0,0 +1,699 @@
|
||||
/* |
||||
* 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.api.service; |
||||
|
||||
//import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.RERUN;
|
||||
//import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_START;
|
||||
//import static org.mockito.ArgumentMatchers.any;
|
||||
//import static org.mockito.ArgumentMatchers.argThat;
|
||||
//import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
|
||||
//import static org.mockito.Mockito.doReturn;
|
||||
//import static org.mockito.Mockito.times;
|
||||
//import static org.mockito.Mockito.verify;
|
||||
//import static org.mockito.Mockito.when;
|
||||
//
|
||||
//import org.apache.dolphinscheduler.api.dto.workflowInstance.WorkflowExecuteResponse;
|
||||
//import org.apache.dolphinscheduler.api.enums.ExecuteType;
|
||||
//import org.apache.dolphinscheduler.api.enums.Status;
|
||||
//import org.apache.dolphinscheduler.api.exceptions.ServiceException;
|
||||
//import org.apache.dolphinscheduler.api.executor.workflow.instance.WorkflowInstanceExecuteClient;
|
||||
//import org.apache.dolphinscheduler.api.permission.ResourcePermissionCheckService;
|
||||
//import org.apache.dolphinscheduler.api.service.impl.ExecutorServiceImpl;
|
||||
//import org.apache.dolphinscheduler.api.service.impl.ProjectServiceImpl;
|
||||
//import org.apache.dolphinscheduler.common.constants.Constants;
|
||||
//import org.apache.dolphinscheduler.common.enums.CommandType;
|
||||
//import org.apache.dolphinscheduler.common.enums.ComplementDependentMode;
|
||||
//import org.apache.dolphinscheduler.common.enums.ExecutionOrder;
|
||||
//import org.apache.dolphinscheduler.common.enums.FailureStrategy;
|
||||
//import org.apache.dolphinscheduler.common.enums.Priority;
|
||||
//import org.apache.dolphinscheduler.common.enums.ReleaseState;
|
||||
//import org.apache.dolphinscheduler.common.enums.RunMode;
|
||||
//import org.apache.dolphinscheduler.common.enums.TaskDependType;
|
||||
//import org.apache.dolphinscheduler.common.enums.TaskGroupQueueStatus;
|
||||
//import org.apache.dolphinscheduler.common.enums.WarningType;
|
||||
//import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
|
||||
//import org.apache.dolphinscheduler.common.model.Server;
|
||||
//import org.apache.dolphinscheduler.dao.entity.Command;
|
||||
//import org.apache.dolphinscheduler.dao.entity.DependentProcessDefinition;
|
||||
//import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
|
||||
//import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
|
||||
//import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelation;
|
||||
//import org.apache.dolphinscheduler.dao.entity.Project;
|
||||
//import org.apache.dolphinscheduler.dao.entity.Schedule;
|
||||
//import org.apache.dolphinscheduler.dao.entity.TaskGroupQueue;
|
||||
//import org.apache.dolphinscheduler.dao.entity.Tenant;
|
||||
//import org.apache.dolphinscheduler.dao.entity.User;
|
||||
//import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
|
||||
//import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper;
|
||||
//import org.apache.dolphinscheduler.dao.mapper.ProcessTaskRelationMapper;
|
||||
//import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
|
||||
//import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper;
|
||||
//import org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper;
|
||||
//import org.apache.dolphinscheduler.dao.mapper.TaskGroupQueueMapper;
|
||||
//import org.apache.dolphinscheduler.dao.mapper.TenantMapper;
|
||||
//import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
|
||||
//import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils;
|
||||
//import org.apache.dolphinscheduler.registry.api.enums.RegistryNodeType;
|
||||
//import org.apache.dolphinscheduler.service.command.CommandService;
|
||||
//import org.apache.dolphinscheduler.service.process.ProcessService;
|
||||
//import org.apache.dolphinscheduler.service.process.TriggerRelationService;
|
||||
//
|
||||
//import java.util.ArrayList;
|
||||
//import java.util.Collections;
|
||||
//import java.util.Date;
|
||||
//import java.util.HashMap;
|
||||
//import java.util.LinkedList;
|
||||
//import java.util.List;
|
||||
//import java.util.Map;
|
||||
//import java.util.Optional;
|
||||
//
|
||||
//import org.assertj.core.util.Lists;
|
||||
//import org.junit.jupiter.api.Assertions;
|
||||
//import org.junit.jupiter.api.BeforeEach;
|
||||
//import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith; |
||||
import org.mockito.junit.jupiter.MockitoExtension; |
||||
import org.mockito.junit.jupiter.MockitoSettings; |
||||
import org.mockito.quality.Strictness; |
||||
|
||||
/** |
||||
* executor service 2 test |
||||
*/ |
||||
@ExtendWith(MockitoExtension.class) |
||||
@MockitoSettings(strictness = Strictness.LENIENT) |
||||
public class WorkflowInstanceExecuteFunctionServiceTest { |
||||
|
||||
// private static final Logger logger = LoggerFactory.getLogger(WorkflowInstanceExecuteFunctionServiceTest.class);
|
||||
//
|
||||
// @Mock
|
||||
// private ResourcePermissionCheckService resourcePermissionCheckService;
|
||||
//
|
||||
// @InjectMocks
|
||||
// private ExecutorServiceImpl executorService;
|
||||
//
|
||||
// @Mock
|
||||
// private ProcessService processService;
|
||||
//
|
||||
// @Mock
|
||||
// private CommandService commandService;
|
||||
//
|
||||
// @Mock
|
||||
// private WorkerGroupService workerGroupService;
|
||||
//
|
||||
// @Mock
|
||||
// private ProcessDefinitionMapper processDefinitionMapper;
|
||||
//
|
||||
// @Mock
|
||||
// private ProcessTaskRelationMapper processTaskRelationMapper;
|
||||
//
|
||||
// @Mock
|
||||
// private TaskDefinitionMapper taskDefinitionMapper;
|
||||
//
|
||||
// @Mock
|
||||
// private TaskDefinitionLogMapper taskDefinitionLogMapper;
|
||||
//
|
||||
// @Mock
|
||||
// private ProjectMapper projectMapper;
|
||||
//
|
||||
// @Mock
|
||||
// private TenantMapper tenantMapper;
|
||||
//
|
||||
// @Mock
|
||||
// private ProjectServiceImpl projectService;
|
||||
//
|
||||
// @Mock
|
||||
// private MonitorService monitorService;
|
||||
//
|
||||
// @Mock
|
||||
// private TaskGroupQueueMapper taskGroupQueueMapper;
|
||||
//
|
||||
// @Mock
|
||||
// private ProcessInstanceMapper processInstanceMapper;
|
||||
//
|
||||
// @Mock
|
||||
// private TriggerRelationService triggerRelationService;
|
||||
//
|
||||
// @Mock
|
||||
// private WorkflowInstanceExecuteClient workflowInstanceExecuteClient;
|
||||
//
|
||||
// @Mock
|
||||
// private ProcessInstanceDao processInstanceDao;
|
||||
//
|
||||
// @Mock
|
||||
// private ProcessDefinitionService processDefinitionService;
|
||||
//
|
||||
// @Mock
|
||||
// private ProcessLineageService processLineageService;
|
||||
//
|
||||
// private int processDefinitionId = 1;
|
||||
//
|
||||
// private int processDefinitionVersion = 1;
|
||||
//
|
||||
// private long processDefinitionCode = 1L;
|
||||
//
|
||||
// private int processInstanceId = 1;
|
||||
//
|
||||
// private String tenantCode = "root";
|
||||
//
|
||||
// private int userId = 1;
|
||||
//
|
||||
// private int taskQueueId = 1;
|
||||
//
|
||||
// private ProcessDefinition processDefinition = new ProcessDefinition();
|
||||
//
|
||||
// private ProcessInstance processInstance = new ProcessInstance();
|
||||
//
|
||||
// private TaskGroupQueue taskGroupQueue = new TaskGroupQueue();
|
||||
//
|
||||
// private List<ProcessTaskRelation> processTaskRelations = new ArrayList<>();
|
||||
//
|
||||
// private User loginUser = new User();
|
||||
//
|
||||
// private long projectCode = 1L;
|
||||
//
|
||||
// private String projectName = "projectName";
|
||||
//
|
||||
// private Project project = new Project();
|
||||
//
|
||||
// private String cronTime;
|
||||
//
|
||||
// @BeforeEach
|
||||
// public void init() {
|
||||
// // user
|
||||
// loginUser.setId(userId);
|
||||
//
|
||||
// // processDefinition
|
||||
// processDefinition.setId(processDefinitionId);
|
||||
// processDefinition.setReleaseState(ReleaseState.ONLINE);
|
||||
// processDefinition.setUserId(userId);
|
||||
// processDefinition.setVersion(1);
|
||||
// processDefinition.setCode(1L);
|
||||
// processDefinition.setProjectCode(projectCode);
|
||||
//
|
||||
// // processInstance
|
||||
// processInstance.setId(processInstanceId);
|
||||
// processInstance.setProjectCode(projectCode);
|
||||
// processInstance.setState(WorkflowExecutionStatus.FAILURE);
|
||||
// processInstance.setExecutorId(userId);
|
||||
// processInstance.setHost("127.0.0.1:5678");
|
||||
// processInstance.setProcessDefinitionVersion(1);
|
||||
// processInstance.setProcessDefinitionCode(1L);
|
||||
//
|
||||
// // project
|
||||
// project.setCode(projectCode);
|
||||
// project.setName(projectName);
|
||||
//
|
||||
// // taskGroupQueue
|
||||
// taskGroupQueue.setId(taskQueueId);
|
||||
// taskGroupQueue.setStatus(TaskGroupQueueStatus.WAIT_QUEUE);
|
||||
// taskGroupQueue.setProcessId(processInstanceId);
|
||||
//
|
||||
// // cronRangeTime
|
||||
// cronTime = "2020-01-01 00:00:00,2020-01-31 23:00:00";
|
||||
//
|
||||
// // processTaskRelations
|
||||
// ProcessTaskRelation processTaskRelation1 = new ProcessTaskRelation();
|
||||
// processTaskRelation1.setPostTaskCode(123456789L);
|
||||
// ProcessTaskRelation processTaskRelation2 = new ProcessTaskRelation();
|
||||
// processTaskRelation2.setPostTaskCode(987654321L);
|
||||
// processTaskRelations.add(processTaskRelation1);
|
||||
// processTaskRelations.add(processTaskRelation2);
|
||||
//
|
||||
// // mock
|
||||
// Mockito.when(projectMapper.queryByCode(projectCode)).thenReturn(project);
|
||||
// Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_START))
|
||||
// .thenReturn(checkProjectAndAuth());
|
||||
// Mockito.when(processDefinitionMapper.queryByCode(processDefinitionCode)).thenReturn(this.processDefinition);
|
||||
// Mockito.when(processService.getTenantForProcess(tenantCode, userId)).thenReturn(tenantCode);
|
||||
// doReturn(1).when(commandService).createCommand(argThat(c -> c.getId() == null));
|
||||
// doReturn(0).when(commandService).createCommand(argThat(c -> c.getId() != null));
|
||||
// Mockito.when(monitorService.listServer(RegistryNodeType.MASTER)).thenReturn(getMasterServersList());
|
||||
// Mockito.when(processService.findProcessInstanceDetailById(processInstanceId))
|
||||
// .thenReturn(Optional.ofNullable(processInstance));
|
||||
// Mockito.when(processService.findProcessDefinition(1L, 1)).thenReturn(this.processDefinition);
|
||||
// Mockito.when(taskGroupQueueMapper.selectById(1)).thenReturn(taskGroupQueue);
|
||||
// Mockito.when(processInstanceMapper.selectById(1)).thenReturn(processInstance);
|
||||
// Mockito.when(triggerRelationService.saveProcessInstanceTrigger(Mockito.any(), Mockito.any()))
|
||||
// .thenReturn(1);
|
||||
// Mockito.when(processService.findRelationByCode(processDefinitionCode, processDefinitionVersion))
|
||||
// .thenReturn(processTaskRelations);
|
||||
// }
|
||||
//
|
||||
// /**
|
||||
// * not complement
|
||||
// */
|
||||
// @Test
|
||||
// public void testNoComplement() {
|
||||
//
|
||||
// Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
|
||||
// .thenReturn(zeroSchedulerList());
|
||||
// Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant());
|
||||
// Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
|
||||
// processDefinitionCode,
|
||||
// "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
|
||||
// CommandType.START_PROCESS,
|
||||
// null, null,
|
||||
// null, null, null,
|
||||
// RunMode.RUN_MODE_SERIAL,
|
||||
// Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 10, null, null,
|
||||
// Constants.DRY_RUN_FLAG_NO,
|
||||
// Constants.TEST_FLAG_NO,
|
||||
// ComplementDependentMode.OFF_MODE, null,
|
||||
// false,
|
||||
// ExecutionOrder.DESC_ORDER);
|
||||
// Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
|
||||
// verify(commandService, times(1)).createCommand(any(Command.class));
|
||||
//
|
||||
// }
|
||||
//
|
||||
// /**
|
||||
// * not complement
|
||||
// */
|
||||
// @Test
|
||||
// public void testComplementWithStartNodeList() {
|
||||
//
|
||||
// Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
|
||||
// .thenReturn(zeroSchedulerList());
|
||||
// Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant());
|
||||
// Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
|
||||
// processDefinitionCode,
|
||||
// "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
|
||||
// CommandType.START_PROCESS,
|
||||
// null, "123456789,987654321",
|
||||
// null, null, null,
|
||||
// RunMode.RUN_MODE_SERIAL,
|
||||
// Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, null,
|
||||
// Constants.DRY_RUN_FLAG_NO,
|
||||
// Constants.TEST_FLAG_NO,
|
||||
// ComplementDependentMode.OFF_MODE, null,
|
||||
// false,
|
||||
// ExecutionOrder.DESC_ORDER);
|
||||
// Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
|
||||
// verify(commandService, times(1)).createCommand(any(Command.class));
|
||||
//
|
||||
// }
|
||||
//
|
||||
// @Test
|
||||
// public void testComplementWithOldStartNodeList() {
|
||||
// Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
|
||||
// .thenReturn(zeroSchedulerList());
|
||||
// Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant());
|
||||
// Map<String, Object> result = new HashMap<>();
|
||||
// try {
|
||||
// result = executorService.execProcessInstance(loginUser, projectCode,
|
||||
// processDefinitionCode,
|
||||
// "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
|
||||
// CommandType.START_PROCESS,
|
||||
// null, "1123456789,987654321",
|
||||
// null, null, null,
|
||||
// RunMode.RUN_MODE_SERIAL,
|
||||
// Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, 0,
|
||||
// Constants.DRY_RUN_FLAG_NO,
|
||||
// Constants.TEST_FLAG_NO,
|
||||
// ComplementDependentMode.OFF_MODE, null,
|
||||
// false,
|
||||
// ExecutionOrder.DESC_ORDER);
|
||||
// } catch (ServiceException e) {
|
||||
// Assertions.assertEquals(Status.START_NODE_NOT_EXIST_IN_LAST_PROCESS.getCode(), e.getCode());
|
||||
// }
|
||||
// }
|
||||
//
|
||||
// @Test
|
||||
// public void testComplementWithDependentMode() {
|
||||
// Schedule schedule = new Schedule();
|
||||
// schedule.setStartTime(new Date());
|
||||
// schedule.setEndTime(new Date());
|
||||
// schedule.setCrontab("0 0 7 * * ? *");
|
||||
// schedule.setFailureStrategy(FailureStrategy.CONTINUE);
|
||||
// schedule.setReleaseState(ReleaseState.OFFLINE);
|
||||
// schedule.setWarningType(WarningType.NONE);
|
||||
// schedule.setCreateTime(new Date());
|
||||
// schedule.setUpdateTime(new Date());
|
||||
// List<Schedule> schedules = Lists.newArrayList(schedule);
|
||||
// Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(
|
||||
// processDefinitionCode))
|
||||
// .thenReturn(schedules);
|
||||
//
|
||||
// DependentProcessDefinition dependentProcessDefinition = new DependentProcessDefinition();
|
||||
// dependentProcessDefinition.setProcessDefinitionCode(2);
|
||||
// dependentProcessDefinition.setProcessDefinitionVersion(1);
|
||||
// dependentProcessDefinition.setTaskDefinitionCode(1);
|
||||
// dependentProcessDefinition.setWorkerGroup(WorkerGroupUtils.getDefaultWorkerGroup());
|
||||
// dependentProcessDefinition.setTaskParams(
|
||||
// "{\"localParams\":[],\"resourceList\":[],\"dependence\":{\"relation\":\"AND\",\"dependTaskList\":[{\"relation\":\"AND\",\"dependItemList\":[{\"depTaskCode\":2,\"status\":\"SUCCESS\"}]}]},\"conditionResult\":{\"successNode\":[1],\"failedNode\":[1]}}");
|
||||
// Mockito.when(processLineageService.queryDownstreamDependentProcessDefinitions(processDefinitionCode))
|
||||
// .thenReturn(Lists.newArrayList(dependentProcessDefinition));
|
||||
//
|
||||
// Map<Long, String> processDefinitionWorkerGroupMap = new HashMap<>();
|
||||
// processDefinitionWorkerGroupMap.put(1L, WorkerGroupUtils.getDefaultWorkerGroup());
|
||||
// Mockito.when(workerGroupService.queryWorkerGroupByProcessDefinitionCodes(Lists.newArrayList(1L)))
|
||||
// .thenReturn(processDefinitionWorkerGroupMap);
|
||||
//
|
||||
// Command command = new Command();
|
||||
// command.setId(1);
|
||||
// command.setCommandType(CommandType.COMPLEMENT_DATA);
|
||||
// command.setCommandParam(
|
||||
// "{\"StartNodeList\":\"1\",\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31
|
||||
// 23:00:00\"}");
|
||||
// command.setWorkerGroup(WorkerGroupUtils.getDefaultWorkerGroup());
|
||||
// command.setProcessDefinitionCode(processDefinitionCode);
|
||||
// command.setExecutorId(1);
|
||||
//
|
||||
// // not enable allLevelDependent
|
||||
// int count = executorService.createComplementDependentCommand(schedules, command, false);
|
||||
// Assertions.assertEquals(1, count);
|
||||
//
|
||||
// // enable allLevelDependent
|
||||
// DependentProcessDefinition childDependent = new DependentProcessDefinition();
|
||||
// childDependent.setProcessDefinitionCode(3);
|
||||
// childDependent.setProcessDefinitionVersion(1);
|
||||
// childDependent.setTaskDefinitionCode(4);
|
||||
// childDependent.setWorkerGroup(WorkerGroupUtils.getDefaultWorkerGroup());
|
||||
// childDependent.setTaskParams(
|
||||
// "{\"localParams\":[],\"resourceList\":[],\"dependence\":{\"relation\":\"AND\",\"dependTaskList\":[{\"relation\":\"AND\",\"dependItemList\":[{\"depTaskCode\":3,\"status\":\"SUCCESS\"}]}]},\"conditionResult\":{\"successNode\":[1],\"failedNode\":[1]}}");
|
||||
// Mockito.when(processLineageService.queryDownstreamDependentProcessDefinitions(
|
||||
// dependentProcessDefinition.getProcessDefinitionCode())).thenReturn(Lists.newArrayList(childDependent))
|
||||
// .thenReturn(Lists.newArrayList());
|
||||
// int allLevelDependentCount = executorService.createComplementDependentCommand(schedules, command, true);
|
||||
// Assertions.assertEquals(2, allLevelDependentCount);
|
||||
// }
|
||||
//
|
||||
// /**
|
||||
// * date error
|
||||
// */
|
||||
// @Test
|
||||
// public void testDateError() {
|
||||
//
|
||||
// Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
|
||||
// .thenReturn(zeroSchedulerList());
|
||||
// Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant());
|
||||
// Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
|
||||
// processDefinitionCode,
|
||||
// "{\"complementStartDate\":\"2022-01-07 12:12:12\",\"complementEndDate\":\"2022-01-06 12:12:12\"}",
|
||||
// CommandType.COMPLEMENT_DATA,
|
||||
// null, null,
|
||||
// null, null, null,
|
||||
// RunMode.RUN_MODE_SERIAL,
|
||||
// Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, 2,
|
||||
// Constants.DRY_RUN_FLAG_NO,
|
||||
// Constants.TEST_FLAG_NO,
|
||||
// ComplementDependentMode.OFF_MODE, null,
|
||||
// false,
|
||||
// ExecutionOrder.DESC_ORDER);
|
||||
// Assertions.assertEquals(Status.START_PROCESS_INSTANCE_ERROR, result.get(Constants.STATUS));
|
||||
// verify(commandService, times(0)).createCommand(any(Command.class));
|
||||
// }
|
||||
//
|
||||
// /**
|
||||
// * serial
|
||||
// */
|
||||
// @Test
|
||||
// public void testSerial() {
|
||||
//
|
||||
// Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
|
||||
// .thenReturn(zeroSchedulerList());
|
||||
// Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant());
|
||||
// Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
|
||||
// processDefinitionCode,
|
||||
// "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
|
||||
// CommandType.COMPLEMENT_DATA,
|
||||
// null, null,
|
||||
// null, null, null,
|
||||
// RunMode.RUN_MODE_SERIAL,
|
||||
// Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, null,
|
||||
// Constants.DRY_RUN_FLAG_NO,
|
||||
// Constants.TEST_FLAG_NO,
|
||||
// ComplementDependentMode.OFF_MODE, null,
|
||||
// false,
|
||||
// ExecutionOrder.DESC_ORDER);
|
||||
// Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
|
||||
// verify(commandService, times(1)).createCommand(any(Command.class));
|
||||
// }
|
||||
//
|
||||
// /**
|
||||
// * without schedule
|
||||
// */
|
||||
// @Test
|
||||
// public void testParallelWithOutSchedule() {
|
||||
//
|
||||
// Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
|
||||
// .thenReturn(zeroSchedulerList());
|
||||
// Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant());
|
||||
// Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
|
||||
// processDefinitionCode,
|
||||
// "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
|
||||
// CommandType.COMPLEMENT_DATA,
|
||||
// null, null,
|
||||
// null, null, null,
|
||||
// RunMode.RUN_MODE_PARALLEL,
|
||||
// Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, 2,
|
||||
// Constants.DRY_RUN_FLAG_NO,
|
||||
// Constants.TEST_FLAG_NO,
|
||||
// ComplementDependentMode.OFF_MODE, null,
|
||||
// false,
|
||||
// ExecutionOrder.DESC_ORDER);
|
||||
//
|
||||
// Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
|
||||
// verify(commandService, times(2)).createCommand(any(Command.class));
|
||||
// }
|
||||
//
|
||||
// /**
|
||||
// * with schedule
|
||||
// */
|
||||
// @Test
|
||||
// public void testParallelWithSchedule() {
|
||||
//
|
||||
// Mockito.when(processService.queryReleaseSchedulerListByProcessDefinitionCode(processDefinitionCode))
|
||||
// .thenReturn(oneSchedulerList());
|
||||
// Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant());
|
||||
// Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
|
||||
// processDefinitionCode,
|
||||
// "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
|
||||
// CommandType.COMPLEMENT_DATA,
|
||||
// null, null,
|
||||
// null, null, null,
|
||||
// RunMode.RUN_MODE_PARALLEL,
|
||||
// Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, 15,
|
||||
// Constants.DRY_RUN_FLAG_NO,
|
||||
// Constants.TEST_FLAG_NO,
|
||||
// ComplementDependentMode.OFF_MODE, null,
|
||||
// false,
|
||||
// ExecutionOrder.DESC_ORDER);
|
||||
// Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
|
||||
// verify(commandService, times(15)).createCommand(any(Command.class));
|
||||
//
|
||||
// }
|
||||
//
|
||||
// @Test
|
||||
// public void testNoMasterServers() {
|
||||
// Mockito.when(monitorService.listServer(RegistryNodeType.MASTER)).thenReturn(new ArrayList<>());
|
||||
//
|
||||
// Assertions.assertThrows(ServiceException.class, () -> executorService.execProcessInstance(
|
||||
// loginUser,
|
||||
// projectCode,
|
||||
// processDefinitionCode,
|
||||
// "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
|
||||
// CommandType.COMPLEMENT_DATA,
|
||||
// null,
|
||||
// null,
|
||||
// null,
|
||||
// null,
|
||||
// null,
|
||||
// RunMode.RUN_MODE_PARALLEL,
|
||||
// Priority.LOW,
|
||||
// WorkerGroupUtils.getDefaultWorkerGroup(),
|
||||
// tenantCode,
|
||||
// 100L,
|
||||
// 110,
|
||||
// null,
|
||||
// null,
|
||||
// Constants.DRY_RUN_FLAG_NO,
|
||||
// Constants.TEST_FLAG_NO,
|
||||
// ComplementDependentMode.OFF_MODE, null,
|
||||
// false,
|
||||
// ExecutionOrder.DESC_ORDER));
|
||||
// }
|
||||
//
|
||||
// @Test
|
||||
// public void testExecuteRepeatRunning() {
|
||||
// when(commandService.verifyIsNeedCreateCommand(any(Command.class))).thenReturn(true);
|
||||
// when(projectService.checkProjectAndAuth(loginUser, project, projectCode, RERUN))
|
||||
// .thenReturn(checkProjectAndAuth());
|
||||
// when(processInstanceDao.queryOptionalById(processInstanceId)).thenReturn(Optional.of(processInstance));
|
||||
// when(processDefinitionService.queryWorkflowDefinitionThrowExceptionIfNotFound(processDefinitionCode,
|
||||
// processDefinitionVersion)).thenReturn(processDefinition);
|
||||
// Assertions.assertDoesNotThrow(() -> {
|
||||
// executorService.controlWorkflowInstance(loginUser, processInstanceId, ExecuteType.REPEAT_RUNNING);
|
||||
// });
|
||||
// }
|
||||
//
|
||||
// @Test
|
||||
// public void testOfTestRun() {
|
||||
// Mockito.when(commandService.verifyIsNeedCreateCommand(any(Command.class))).thenReturn(true);
|
||||
// Mockito.when(projectService.checkProjectAndAuth(loginUser, project, projectCode, RERUN))
|
||||
// .thenReturn(checkProjectAndAuth());
|
||||
// Mockito.when(tenantMapper.queryByTenantCode(tenantCode)).thenReturn(new Tenant());
|
||||
// Map<String, Object> result = executorService.execProcessInstance(loginUser, projectCode,
|
||||
// processDefinitionCode,
|
||||
// "{\"complementStartDate\":\"2020-01-01 00:00:00\",\"complementEndDate\":\"2020-01-31 23:00:00\"}",
|
||||
// CommandType.COMPLEMENT_DATA,
|
||||
// null, null,
|
||||
// null, null, 0,
|
||||
// RunMode.RUN_MODE_PARALLEL,
|
||||
// Priority.LOW, WorkerGroupUtils.getDefaultWorkerGroup(), tenantCode, 100L, 110, null, 15,
|
||||
// Constants.DRY_RUN_FLAG_NO,
|
||||
// Constants.TEST_FLAG_YES,
|
||||
// ComplementDependentMode.OFF_MODE, null,
|
||||
// false,
|
||||
// ExecutionOrder.DESC_ORDER);
|
||||
// Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
|
||||
// }
|
||||
//
|
||||
// @Test
|
||||
// public void testStartCheckByProcessDefinedCode() {
|
||||
// List<Long> ids = Lists.newArrayList(1L);
|
||||
// when(processService.findAllSubWorkflowDefinitionCode(1)).thenReturn(ids);
|
||||
//
|
||||
// List<ProcessDefinition> processDefinitionList = new ArrayList<>();
|
||||
// ProcessDefinition processDefinition = new ProcessDefinition();
|
||||
// processDefinition.setId(1);
|
||||
// processDefinition.setReleaseState(ReleaseState.ONLINE);
|
||||
// processDefinitionList.add(processDefinition);
|
||||
// Mockito.when(processDefinitionMapper.queryDefinitionListByIdList(new Integer[ids.size()]))
|
||||
// .thenReturn(processDefinitionList);
|
||||
//
|
||||
// Map<String, Object> result = executorService.startCheckByProcessDefinedCode(1L);
|
||||
// Assertions.assertEquals(Status.SUCCESS, result.get(Constants.STATUS));
|
||||
// }
|
||||
//
|
||||
// private List<Server> getMasterServersList() {
|
||||
// List<Server> masterServerList = new ArrayList<>();
|
||||
// Server masterServer1 = new Server();
|
||||
// masterServer1.setId(1);
|
||||
// masterServer1.setHost("192.168.220.188");
|
||||
// masterServer1.setPort(1121);
|
||||
// masterServerList.add(masterServer1);
|
||||
//
|
||||
// Server masterServer2 = new Server();
|
||||
// masterServer2.setId(2);
|
||||
// masterServer2.setHost("192.168.220.189");
|
||||
// masterServer2.setPort(1122);
|
||||
// masterServerList.add(masterServer2);
|
||||
//
|
||||
// return masterServerList;
|
||||
// }
|
||||
//
|
||||
// private List zeroSchedulerList() {
|
||||
// return Collections.EMPTY_LIST;
|
||||
// }
|
||||
//
|
||||
// private List<Schedule> oneSchedulerList() {
|
||||
// List<Schedule> schedulerList = new LinkedList<>();
|
||||
// Schedule schedule = new Schedule();
|
||||
// schedule.setCrontab("0 0 0 1/2 * ?");
|
||||
// schedulerList.add(schedule);
|
||||
// return schedulerList;
|
||||
// }
|
||||
//
|
||||
// private Map<String, Object> checkProjectAndAuth() {
|
||||
// Map<String, Object> result = new HashMap<>();
|
||||
// result.put(Constants.STATUS, Status.SUCCESS);
|
||||
// return result;
|
||||
// }
|
||||
//
|
||||
// @Test
|
||||
// public void testCreateComplementToParallel() {
|
||||
// List<String> result = new ArrayList<>();
|
||||
// int expectedParallelismNumber = 3;
|
||||
// LinkedList<Integer> listDate = new LinkedList<>();
|
||||
// listDate.add(0);
|
||||
// listDate.add(1);
|
||||
// listDate.add(2);
|
||||
// listDate.add(3);
|
||||
// listDate.add(4);
|
||||
//
|
||||
// int listDateSize = listDate.size();
|
||||
// int createCount = Math.min(listDate.size(), expectedParallelismNumber);
|
||||
// logger.info("In parallel mode, current expectedParallelismNumber:{}", createCount);
|
||||
//
|
||||
// int itemsPerCommand = (listDateSize / createCount);
|
||||
// int remainingItems = (listDateSize % createCount);
|
||||
// int startDateIndex = 0;
|
||||
// int endDateIndex = 0;
|
||||
//
|
||||
// for (int i = 1; i <= createCount; i++) {
|
||||
// int extra = (i <= remainingItems) ? 1 : 0;
|
||||
// int singleCommandItems = (itemsPerCommand + extra);
|
||||
//
|
||||
// if (i == 1) {
|
||||
// endDateIndex += singleCommandItems - 1;
|
||||
// } else {
|
||||
// startDateIndex = endDateIndex + 1;
|
||||
// endDateIndex += singleCommandItems;
|
||||
// }
|
||||
//
|
||||
// logger.info("startDate:{}, endDate:{}", listDate.get(startDateIndex), listDate.get(endDateIndex));
|
||||
// result.add(listDate.get(startDateIndex) + "," + listDate.get(endDateIndex));
|
||||
// }
|
||||
//
|
||||
// Assertions.assertEquals("0,1", result.get(0));
|
||||
// Assertions.assertEquals("2,3", result.get(1));
|
||||
// Assertions.assertEquals("4,4", result.get(2));
|
||||
// }
|
||||
//
|
||||
// @Test
|
||||
// public void testExecuteTask() {
|
||||
// String startNodeList = "1234567870";
|
||||
// TaskDependType taskDependType = TaskDependType.TASK_ONLY;
|
||||
//
|
||||
// ProcessInstance processInstanceMock = Mockito.mock(ProcessInstance.class, RETURNS_DEEP_STUBS);
|
||||
// Mockito.when(processService.findProcessInstanceDetailById(processInstanceId))
|
||||
// .thenReturn(Optional.ofNullable(processInstanceMock));
|
||||
//
|
||||
// ProcessDefinition processDefinition = new ProcessDefinition();
|
||||
// processDefinition.setProjectCode(projectCode);
|
||||
// Mockito.when(processService.findProcessDefinition(Mockito.anyLong(), Mockito.anyInt()))
|
||||
// .thenReturn(processDefinition);
|
||||
//
|
||||
// Mockito.when(processService.getTenantForProcess(Mockito.anyString(), Mockito.anyInt())).thenReturn(tenantCode);
|
||||
//
|
||||
// when(processInstanceMock.getState().isFinished()).thenReturn(false);
|
||||
// WorkflowExecuteResponse responseInstanceIsNotFinished =
|
||||
// executorService.executeTask(loginUser, projectCode, processInstanceId, startNodeList, taskDependType);
|
||||
// Assertions.assertEquals(Status.WORKFLOW_INSTANCE_IS_NOT_FINISHED.getCode(),
|
||||
// responseInstanceIsNotFinished.getCode());
|
||||
//
|
||||
// when(processInstanceMock.getState().isFinished()).thenReturn(true);
|
||||
// WorkflowExecuteResponse responseStartNodeListError =
|
||||
// executorService.executeTask(loginUser, projectCode, processInstanceId, "1234567870,", taskDependType);
|
||||
// Assertions.assertEquals(Status.REQUEST_PARAMS_NOT_VALID_ERROR.getCode(), responseStartNodeListError.getCode());
|
||||
//
|
||||
// Mockito.when(taskDefinitionLogMapper.queryMaxVersionForDefinition(Mockito.anyLong())).thenReturn(null);
|
||||
// WorkflowExecuteResponse responseNotDefineTask =
|
||||
// executorService.executeTask(loginUser, projectCode, processInstanceId, startNodeList, taskDependType);
|
||||
// Assertions.assertEquals(Status.EXECUTE_NOT_DEFINE_TASK.getCode(), responseNotDefineTask.getCode());
|
||||
//
|
||||
// Mockito.when(taskDefinitionLogMapper.queryMaxVersionForDefinition(Mockito.anyLong())).thenReturn(1);
|
||||
// Mockito.when(commandService.verifyIsNeedCreateCommand(any())).thenReturn(true);
|
||||
// WorkflowExecuteResponse responseSuccess =
|
||||
// executorService.executeTask(loginUser, projectCode, processInstanceId, startNodeList, taskDependType);
|
||||
// Assertions.assertEquals(Status.SUCCESS.getCode(), responseSuccess.getCode());
|
||||
//
|
||||
// }
|
||||
|
||||
} |
@ -0,0 +1,32 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?> |
||||
<!-- |
||||
~ 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. |
||||
~ |
||||
--> |
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" |
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> |
||||
<modelVersion>4.0.0</modelVersion> |
||||
<parent> |
||||
<groupId>org.apache.dolphinscheduler</groupId> |
||||
<artifactId>dolphinscheduler</artifactId> |
||||
<version>dev-SNAPSHOT</version> |
||||
</parent> |
||||
|
||||
<artifactId>dolphinscheduler-eventbus</artifactId> |
||||
|
||||
</project> |
@ -0,0 +1,56 @@
|
||||
/* |
||||
* 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.eventbus; |
||||
|
||||
import java.util.concurrent.Delayed; |
||||
import java.util.concurrent.TimeUnit; |
||||
|
||||
/** |
||||
* The abstract class of delay event, the event will be triggered after the delay time. |
||||
* <p> You can extend this class to implement your own delay event. |
||||
*/ |
||||
public abstract class AbstractDelayEvent implements IEvent, Delayed { |
||||
|
||||
protected long delayTime; |
||||
|
||||
protected long triggerTimeInMillis; |
||||
|
||||
public AbstractDelayEvent() { |
||||
this(0); |
||||
} |
||||
|
||||
public AbstractDelayEvent(long delayTime) { |
||||
if (delayTime == 0) { |
||||
this.triggerTimeInMillis = System.currentTimeMillis(); |
||||
} else { |
||||
this.triggerTimeInMillis = System.currentTimeMillis() + delayTime; |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public long getDelay(TimeUnit unit) { |
||||
long delay = triggerTimeInMillis - System.currentTimeMillis(); |
||||
return unit.convert(delay, TimeUnit.MILLISECONDS); |
||||
} |
||||
|
||||
@Override |
||||
public int compareTo(Delayed other) { |
||||
return Long.compare(this.triggerTimeInMillis, ((AbstractDelayEvent) other).triggerTimeInMillis); |
||||
} |
||||
|
||||
} |
@ -0,0 +1,44 @@
|
||||
/* |
||||
* 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.eventbus; |
||||
|
||||
import java.util.Optional; |
||||
import java.util.concurrent.DelayQueue; |
||||
|
||||
/** |
||||
* The event bus that supports delay event. |
||||
*/ |
||||
public abstract class AbstractDelayEventBus<T extends AbstractDelayEvent> implements IEventBus<T> { |
||||
|
||||
protected final DelayQueue<T> delayEventQueue = new DelayQueue<>(); |
||||
|
||||
@Override |
||||
public void publish(final T event) { |
||||
delayEventQueue.add(event); |
||||
} |
||||
|
||||
@Override |
||||
public Optional<T> poll() { |
||||
return Optional.ofNullable(delayEventQueue.poll()); |
||||
} |
||||
|
||||
@Override |
||||
public boolean isEmpty() { |
||||
return delayEventQueue.isEmpty(); |
||||
} |
||||
} |
@ -0,0 +1,50 @@
|
||||
/* |
||||
* 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.extract.master; |
||||
|
||||
import org.apache.dolphinscheduler.extract.base.RpcMethod; |
||||
import org.apache.dolphinscheduler.extract.base.RpcService; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionDispatchEvent; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionFailedEvent; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionKilledEvent; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionPausedEvent; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionRunningEvent; |
||||
import org.apache.dolphinscheduler.extract.master.transportor.TaskExecutionSuccessEvent; |
||||
|
||||
@RpcService |
||||
public interface ITaskExecutionEventListener { |
||||
|
||||
@RpcMethod |
||||
void onTaskInstanceDispatched(final TaskExecutionDispatchEvent taskExecutionDispatchEvent); |
||||
|
||||
@RpcMethod |
||||
void onTaskInstanceExecutionRunning(final TaskExecutionRunningEvent taskInstanceExecutionRunningEvent); |
||||
|
||||
@RpcMethod |
||||
void onTaskInstanceExecutionSuccess(final TaskExecutionSuccessEvent taskInstanceExecutionSuccessEvent); |
||||
|
||||
@RpcMethod |
||||
void onTaskInstanceExecutionFailed(final TaskExecutionFailedEvent taskInstanceExecutionFailedEvent); |
||||
|
||||
@RpcMethod |
||||
void onTaskInstanceExecutionKilled(final TaskExecutionKilledEvent taskInstanceExecutionKilledEvent); |
||||
|
||||
@RpcMethod |
||||
void onTaskInstanceExecutionPaused(final TaskExecutionPausedEvent taskInstanceExecutionPausedEvent); |
||||
|
||||
} |
24
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ITaskInstanceExecutionEventListener.java → dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/IWorkflowInstanceController.java
24
dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/ITaskInstanceExecutionEventListener.java → dolphinscheduler-extract/dolphinscheduler-extract-master/src/main/java/org/apache/dolphinscheduler/extract/master/IWorkflowInstanceController.java
@ -0,0 +1,41 @@
|
||||
/* |
||||
* 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.extract.master.command; |
||||
|
||||
import org.apache.dolphinscheduler.plugin.task.api.model.Property; |
||||
|
||||
import java.util.List; |
||||
|
||||
import lombok.Data; |
||||
import lombok.NoArgsConstructor; |
||||
import lombok.experimental.SuperBuilder; |
||||
|
||||
@Data |
||||
@SuperBuilder(toBuilder = true) |
||||
@NoArgsConstructor |
||||
public abstract class AbstractCommandParam implements ICommandParam { |
||||
|
||||
protected boolean subWorkflowInstance; |
||||
|
||||
protected List<Long> startNodes; |
||||
|
||||
protected List<Property> commandParams; |
||||
|
||||
protected String timeZone; |
||||
|
||||
} |
@ -0,0 +1,41 @@
|
||||
/* |
||||
* 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.extract.master.command; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
|
||||
import java.util.List; |
||||
|
||||
import lombok.Data; |
||||
import lombok.EqualsAndHashCode; |
||||
import lombok.NoArgsConstructor; |
||||
import lombok.experimental.SuperBuilder; |
||||
|
||||
@Data |
||||
@NoArgsConstructor |
||||
@SuperBuilder(toBuilder = true) |
||||
@EqualsAndHashCode(callSuper = true) |
||||
public class BackfillWorkflowCommandParam extends AbstractCommandParam { |
||||
|
||||
private List<String> backfillTimeList; |
||||
|
||||
@Override |
||||
public CommandType getCommandType() { |
||||
return CommandType.COMPLEMENT_DATA; |
||||
} |
||||
} |
@ -0,0 +1,65 @@
|
||||
/* |
||||
* 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.extract.master.command; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
import org.apache.dolphinscheduler.plugin.task.api.model.Property; |
||||
|
||||
import java.util.List; |
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes; |
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo; |
||||
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, visible = true, property = "commandType") |
||||
@JsonSubTypes({ |
||||
@JsonSubTypes.Type(value = ScheduleWorkflowCommandParam.class, name = "SCHEDULER"), |
||||
@JsonSubTypes.Type(value = RunWorkflowCommandParam.class, name = "START_PROCESS"), |
||||
@JsonSubTypes.Type(value = BackfillWorkflowCommandParam.class, name = "COMPLEMENT_DATA"), |
||||
@JsonSubTypes.Type(value = ReRunWorkflowCommandParam.class, name = "REPEAT_RUNNING"), |
||||
@JsonSubTypes.Type(value = RecoverFailureTaskCommandParam.class, name = "START_FAILURE_TASK_PROCESS"), |
||||
@JsonSubTypes.Type(value = WorkflowFailoverCommandParam.class, name = "RECOVER_TOLERANCE_FAULT"), |
||||
}) |
||||
public interface ICommandParam { |
||||
|
||||
/** |
||||
* The task which need to be as the beginning of the workflow. |
||||
*/ |
||||
List<Long> getStartNodes(); |
||||
|
||||
/** |
||||
* The command params. |
||||
*/ |
||||
List<Property> getCommandParams(); |
||||
|
||||
/** |
||||
* Get the time zone. |
||||
* todo: we should remove this field. |
||||
*/ |
||||
String getTimeZone(); |
||||
|
||||
/** |
||||
* Whether the command is used to trigger a sub workflow instance. |
||||
*/ |
||||
boolean isSubWorkflowInstance(); |
||||
|
||||
/** |
||||
* Get the command type. |
||||
*/ |
||||
CommandType getCommandType(); |
||||
|
||||
} |
@ -0,0 +1,39 @@
|
||||
/* |
||||
* 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.extract.master.command; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
|
||||
import lombok.Data; |
||||
import lombok.EqualsAndHashCode; |
||||
import lombok.NoArgsConstructor; |
||||
import lombok.experimental.SuperBuilder; |
||||
|
||||
@Data |
||||
@NoArgsConstructor |
||||
@SuperBuilder(toBuilder = true) |
||||
@EqualsAndHashCode(callSuper = true) |
||||
public class ReRunWorkflowCommandParam extends AbstractCommandParam { |
||||
|
||||
private Integer workflowInstanceId; |
||||
|
||||
@Override |
||||
public CommandType getCommandType() { |
||||
return CommandType.REPEAT_RUNNING; |
||||
} |
||||
} |
@ -0,0 +1,37 @@
|
||||
/* |
||||
* 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.extract.master.command; |
||||
|
||||
import org.apache.dolphinscheduler.common.enums.CommandType; |
||||
|
||||
import lombok.Data; |
||||
import lombok.EqualsAndHashCode; |
||||
import lombok.NoArgsConstructor; |
||||
import lombok.experimental.SuperBuilder; |
||||
|
||||
@Data |
||||
@NoArgsConstructor |
||||
@SuperBuilder(toBuilder = true) |
||||
@EqualsAndHashCode(callSuper = true) |
||||
public class RecoverFailureTaskCommandParam extends AbstractCommandParam { |
||||
|
||||
@Override |
||||
public CommandType getCommandType() { |
||||
return CommandType.START_FAILURE_TASK_PROCESS; |
||||
} |
||||
} |
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in new issue