Browse Source

Refactor logic task (#13948)

3.2.0-release
Wenjun Ruan 2 years ago committed by GitHub
parent
commit
cf91544b99
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/aspect/AccessLogAspect.java
  2. 3
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkFlowLineageServiceImpl.java
  3. 7
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/constants/Constants.java
  4. 8
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java
  5. 3
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java
  6. 3
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessDefinitionDao.java
  7. 2
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessInstanceDao.java
  8. 29
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectDao.java
  9. 3
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskDefinitionDao.java
  10. 2
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskInstanceDao.java
  11. 12
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessDefinitionDaoImpl.java
  12. 17
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessInstanceDaoImpl.java
  13. 40
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectDaoImpl.java
  14. 10
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskDefinitionDaoImpl.java
  15. 10
      dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskInstanceDaoImpl.java
  16. 1
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java
  17. 5
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java
  18. 353
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java
  19. 118
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcher.java
  20. 8
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/enums/ExecutorType.java
  21. 30
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/CommonHostManager.java
  22. 12
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/HostManager.java
  23. 16
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/LowerWeightHostManager.java
  24. 84
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRejectByWorkerEventHandler.java
  25. 24
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java
  26. 12
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java
  27. 8
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java
  28. 3
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java
  29. 26
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/LogicTaskFactoryNotFoundException.java
  30. 30
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/LogicTaskInitializeException.java
  31. 29
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/MasterTaskExecuteException.java
  32. 29
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskDispatchException.java
  33. 30
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecuteRunnableCreateException.java
  34. 26
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecutionContextCreateException.java
  35. 29
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowCreateException.java
  36. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/CacheProcessor.java
  37. 130
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskDispatchProcessor.java
  38. 83
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskKillProcessor.java
  39. 68
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskPauseProcessor.java
  40. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java
  41. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteResultProcessor.java
  42. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteRunningProcessor.java
  43. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteStartProcessor.java
  44. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskForceStartProcessor.java
  45. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskKillResponseProcessor.java
  46. 68
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskRecallProcessor.java
  47. 12
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskUpdateRuntimeProcessor.java
  48. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskWakeupProcessor.java
  49. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowExecutingDataRequestProcessor.java
  50. 4
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowMetricsCleanUpProcessor.java
  51. 2
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java
  52. 14
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java
  53. 10
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRPCServer.java
  54. 3
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRpcClient.java
  55. 132
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java
  56. 46
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueue.java
  57. 99
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java
  58. 54
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterDelayTaskExecuteRunnableDelayQueue.java
  59. 81
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterDelayTaskExecuteRunnableDelayQueueLooper.java
  60. 183
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerBootstrap.java
  61. 89
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecuteRunnableThreadPool.java
  62. 62
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecutorBootstrap.java
  63. 14
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java
  64. 63
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java
  65. 57
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowEventLooper.java
  66. 340
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java
  67. 134
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableFactory.java
  68. 50
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java
  69. 41
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactory.java
  70. 31
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java
  71. 57
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java
  72. 60
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterDelayTaskExecuteRunnable.java
  73. 49
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterDelayTaskExecuteRunnableFactory.java
  74. 46
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java
  75. 132
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueueLooper.java
  76. 30
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunction.java
  77. 66
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunctionImpl.java
  78. 43
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java
  79. 84
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecutionContext.java
  80. 58
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnable.java
  81. 59
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactory.java
  82. 68
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterDelayTaskExecuteRunnable.java
  83. 26
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterDelayTaskExecuteRunnableFactory.java
  84. 173
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecuteRunnable.java
  85. 49
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecuteRunnableFactoryBuilder.java
  86. 47
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutionContextHolder.java
  87. 97
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/PriorityTaskExecuteRunnable.java
  88. 60
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterDelayTaskExecuteRunnable.java
  89. 45
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterDelayTaskExecuteRunnableFactory.java
  90. 43
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecuteRunnable.java
  91. 27
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecuteRunnableFactory.java
  92. 493
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecutionContextFactory.java
  93. 50
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterMessageSender.java
  94. 50
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterMessageSenderManager.java
  95. 70
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskExecuteResultMessageSender.java
  96. 68
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskExecuteRunningMessageSender.java
  97. 64
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskUpdateRuntimeMessageSender.java
  98. 36
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskDispatchOperator.java
  99. 79
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskKillOperator.java
  100. 26
      dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskOperator.java
  101. Some files were not shown because too many files have changed in this diff Show More

4
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/aspect/AccessLogAspect.java

@ -18,6 +18,7 @@
package org.apache.dolphinscheduler.api.aspect;
import org.apache.dolphinscheduler.common.constants.Constants;
import org.apache.dolphinscheduler.common.utils.CodeGenerateUtils;
import org.apache.dolphinscheduler.dao.entity.User;
import org.apache.commons.lang3.StringUtils;
@ -26,7 +27,6 @@ import java.lang.reflect.Method;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Set;
import java.util.UUID;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
@ -79,7 +79,7 @@ public class AccessLogAspect {
Method method = sign.getMethod();
AccessLogAnnotation annotation = method.getAnnotation(AccessLogAnnotation.class);
String traceId = UUID.randomUUID().toString();
String traceId = String.valueOf(CodeGenerateUtils.getInstance().genCode());
// log request
if (!annotation.ignoreRequest()) {

3
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkFlowLineageServiceImpl.java

@ -233,7 +233,8 @@ public class WorkFlowLineageServiceImpl extends BaseServiceImpl implements WorkF
DependentParameters dependentParameters =
JSONUtils.parseObject(taskDefinitionLog.getDependence(), DependentParameters.class);
if (dependentParameters != null) {
List<DependentTaskModel> dependTaskList = dependentParameters.getDependTaskList();
List<DependentTaskModel> dependTaskList =
dependentParameters.getDependTaskList();
if (!CollectionUtils.isEmpty(dependTaskList)) {
for (DependentTaskModel taskModel : dependTaskList) {
List<DependentItem> dependItemList = taskModel.getDependItemList();

7
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/constants/Constants.java

@ -476,13 +476,6 @@ public final class Constants {
public static final String BLOCKING_CONDITION = "blockingCondition";
public static final String ALERT_WHEN_BLOCKING = "alertWhenBlocking";
/**
* master/worker server use for zk
*/
public static final String MASTER_TYPE = "master";
public static final String WORKER_TYPE = "worker";
public static final String DELETE_OP = "delete";
public static final String ADD_OP = "add";
public static final String ALIAS = "alias";
public static final String CONTENT = "content";
public static final String DEPENDENT_SPLIT = ":||";

8
dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/DateUtils.java

@ -584,6 +584,14 @@ public final class DateUtils {
return intervalSeconds - usedTime;
}
public static long getRemainTime(Long baseTime, long intervalSeconds) {
if (baseTime == null) {
return 0;
}
long usedTime = (System.currentTimeMillis() - baseTime) / 1000;
return intervalSeconds - usedTime;
}
/**
* get current time stamp : yyyyMMddHHmmssSSS
*

3
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskInstance.java

@ -321,6 +321,8 @@ public class TaskInstance implements Serializable {
}
public SwitchParameters getSwitchDependency() {
// todo: We need to directly use Jackson to deserialize the taskParam, rather than parse the map and get from
// field.
if (this.switchDependency == null) {
Map<String, Object> taskParamsMap =
JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
@ -336,6 +338,7 @@ public class TaskInstance implements Serializable {
JSONUtils.parseObject(this.getTaskParams(), new TypeReference<Map<String, Object>>() {
});
taskParamsMap.put(Constants.SWITCH_RESULT, JSONUtils.toJsonString(switchDependency));
this.switchDependency = switchDependency;
this.setTaskParams(JSONUtils.toJsonString(taskParamsMap));
}

3
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessDefinitionDao.java

@ -21,6 +21,7 @@ import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.model.PageListingResult;
import java.util.Collection;
import java.util.List;
import java.util.Optional;
@ -53,4 +54,6 @@ public interface ProcessDefinitionDao {
void deleteById(Integer workflowDefinitionId);
void deleteByWorkflowDefinitionCode(long workflowDefinitionCode);
List<ProcessDefinition> queryByCodes(Collection<Long> processDefinitionCodes);
}

2
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProcessInstanceDao.java

@ -87,4 +87,6 @@ public interface ProcessInstanceDao {
* @return process instance
*/
ProcessInstance queryFirstStartProcessInstance(@Param("processDefinitionCode") Long definitionCode);
ProcessInstance findSubProcessInstanceByParentId(Integer processInstanceId, Integer taskInstanceId);
}

29
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/ProjectDao.java

@ -0,0 +1,29 @@
/*
* 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.dao.repository;
import org.apache.dolphinscheduler.dao.entity.Project;
import java.util.Collection;
import java.util.List;
public interface ProjectDao {
List<Project> queryByCodes(Collection<Long> projectCodes);
}

3
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskDefinitionDao.java

@ -19,6 +19,7 @@ package org.apache.dolphinscheduler.dao.repository;
import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
import java.util.Collection;
import java.util.List;
import java.util.Set;
@ -45,4 +46,6 @@ public interface TaskDefinitionDao {
void deleteByWorkflowDefinitionCodeAndVersion(long workflowDefinitionCode, int workflowDefinitionVersion);
void deleteByTaskDefinitionCodes(Set<Long> needToDeleteTaskDefinitionCodes);
List<TaskDefinition> queryByCodes(Collection<Long> taskDefinitionCodes);
}

2
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/TaskInstanceDao.java

@ -56,7 +56,7 @@ public interface TaskInstanceDao {
* @param processInstance process instance
* @return task instance
*/
TaskInstance submitTaskInstanceToDB(TaskInstance taskInstance, ProcessInstance processInstance);
boolean submitTaskInstanceToDB(TaskInstance taskInstance, ProcessInstance processInstance);
/**
* Query list of valid task instance by process instance id

12
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessDefinitionDaoImpl.java

@ -25,7 +25,11 @@ import org.apache.dolphinscheduler.dao.mapper.ProcessDefinitionMapper;
import org.apache.dolphinscheduler.dao.model.PageListingResult;
import org.apache.dolphinscheduler.dao.repository.ProcessDefinitionDao;
import org.apache.commons.collections4.CollectionUtils;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
@ -96,4 +100,12 @@ public class ProcessDefinitionDaoImpl implements ProcessDefinitionDao {
public void deleteByWorkflowDefinitionCode(long workflowDefinitionCode) {
processDefinitionMapper.deleteByCode(workflowDefinitionCode);
}
@Override
public List<ProcessDefinition> queryByCodes(Collection<Long> processDefinitionCodes) {
if (CollectionUtils.isEmpty(processDefinitionCodes)) {
return Collections.emptyList();
}
return processDefinitionMapper.queryByCodes(processDefinitionCodes);
}
}

17
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProcessInstanceDaoImpl.java

@ -19,6 +19,8 @@ package org.apache.dolphinscheduler.dao.repository.impl;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.ProcessInstanceMap;
import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapMapper;
import org.apache.dolphinscheduler.dao.mapper.ProcessInstanceMapper;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.model.DateInterval;
@ -41,6 +43,9 @@ public class ProcessInstanceDaoImpl implements ProcessInstanceDao {
@Autowired
private ProcessInstanceMapper processInstanceMapper;
@Autowired
private ProcessInstanceMapMapper processInstanceMapMapper;
@Override
public int insertProcessInstance(ProcessInstance processInstance) {
return processInstanceMapper.insert(processInstance);
@ -147,4 +152,16 @@ public class ProcessInstanceDaoImpl implements ProcessInstanceDao {
public ProcessInstance queryFirstStartProcessInstance(Long definitionCode) {
return processInstanceMapper.queryFirstStartProcessInstance(definitionCode);
}
@Override
public ProcessInstance findSubProcessInstanceByParentId(Integer processInstanceId, Integer taskInstanceId) {
ProcessInstance processInstance = null;
ProcessInstanceMap processInstanceMap =
processInstanceMapMapper.queryByParentId(processInstanceId, taskInstanceId);
if (processInstanceMap == null || processInstanceMap.getProcessInstanceId() == 0) {
return processInstance;
}
processInstance = queryByWorkflowInstanceId(processInstanceMap.getProcessInstanceId());
return processInstance;
}
}

40
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/ProjectDaoImpl.java

@ -0,0 +1,40 @@
/*
* 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.dao.repository.impl;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.mapper.ProjectMapper;
import org.apache.dolphinscheduler.dao.repository.ProjectDao;
import java.util.Collection;
import java.util.List;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Repository;
@Repository
public class ProjectDaoImpl implements ProjectDao {
@Autowired
private ProjectMapper projectMapper;
@Override
public List<Project> queryByCodes(Collection<Long> projectCodes) {
return projectMapper.queryByCodes(projectCodes);
}
}

10
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskDefinitionDaoImpl.java

@ -30,6 +30,8 @@ import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao;
import org.apache.commons.collections4.CollectionUtils;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -100,4 +102,12 @@ public class TaskDefinitionDaoImpl implements TaskDefinitionDao {
taskDefinitionMapper.deleteByBatchCodes(new ArrayList<>(needToDeleteTaskDefinitionCodes));
}
@Override
public List<TaskDefinition> queryByCodes(Collection<Long> taskDefinitionCodes) {
if (CollectionUtils.isEmpty(taskDefinitionCodes)) {
return Collections.emptyList();
}
return taskDefinitionMapper.queryByCodeList(taskDefinitionCodes);
}
}

10
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/repository/impl/TaskInstanceDaoImpl.java

@ -78,14 +78,14 @@ public class TaskInstanceDaoImpl implements TaskInstanceDao {
}
@Override
public TaskInstance submitTaskInstanceToDB(TaskInstance taskInstance, ProcessInstance processInstance) {
public boolean submitTaskInstanceToDB(TaskInstance taskInstance, ProcessInstance processInstance) {
WorkflowExecutionStatus processInstanceState = processInstance.getState();
if (processInstanceState.isFinished() || processInstanceState == WorkflowExecutionStatus.READY_STOP) {
log.warn("processInstance: {} state was: {}, skip submit this task, taskCode: {}",
processInstance.getId(),
processInstanceState,
taskInstance.getTaskCode());
return null;
return false;
}
if (processInstanceState == WorkflowExecutionStatus.READY_PAUSE) {
taskInstance.setState(TaskExecutionStatus.PAUSE);
@ -99,11 +99,7 @@ public class TaskInstanceDaoImpl implements TaskInstanceDao {
if (taskInstance.getFirstSubmitTime() == null) {
taskInstance.setFirstSubmitTime(taskInstance.getSubmitTime());
}
boolean saveResult = upsertTaskInstance(taskInstance);
if (!saveResult) {
return null;
}
return taskInstance;
return upsertTaskInstance(taskInstance);
}
private TaskExecutionStatus getSubmitTaskState(TaskInstance taskInstance, ProcessInstance processInstance) {

1
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/MasterServer.java

@ -93,7 +93,6 @@ public class MasterServer implements IStoppable {
this.masterRegistryClient.start();
this.masterRegistryClient.setRegistryStoppable(this);
this.masterSchedulerBootstrap.init();
this.masterSchedulerBootstrap.start();
this.eventExecuteService.start();

5
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/config/MasterConfig.java

@ -60,6 +60,11 @@ public class MasterConfig implements Validator {
* Will create two thread poll to execute {@link WorkflowExecuteRunnable} and {@link TaskExecuteRunnable}.
*/
private int execThreads = 10;
// todo: change to sync thread pool/ async thread pool ?
private int masterTaskExecuteThreadPoolSize = Runtime.getRuntime().availableProcessors();
private int masterAsyncTaskStateCheckThreadPoolSize = Runtime.getRuntime().availableProcessors();
/**
* The task dispatch thread pool size.
*/

353
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java

@ -1,353 +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.server.master.consumer;
import org.apache.dolphinscheduler.common.constants.Constants;
import org.apache.dolphinscheduler.common.enums.Flag;
import org.apache.dolphinscheduler.common.enums.TaskEventType;
import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager;
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.dao.utils.TaskCacheUtils;
import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.task.TaskDispatchMessage;
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher;
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
import org.apache.dolphinscheduler.service.exceptions.TaskPriorityQueueException;
import org.apache.dolphinscheduler.service.queue.TaskPriority;
import org.apache.dolphinscheduler.service.queue.TaskPriorityQueue;
import org.apache.commons.collections4.CollectionUtils;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Date;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import javax.annotation.PostConstruct;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
/**
* TaskUpdateQueue consumer
*/
@Component
@Slf4j
public class TaskPriorityQueueConsumer extends BaseDaemonThread {
/**
* taskUpdateQueue
*/
@Autowired
private TaskPriorityQueue<TaskPriority> taskPriorityQueue;
@Autowired
private TaskInstanceDao taskInstanceDao;
/**
* executor dispatcher
*/
@Autowired
private ExecutorDispatcher dispatcher;
/**
* processInstance cache manager
*/
@Autowired
private ProcessInstanceExecCacheManager processInstanceExecCacheManager;
/**
* master config
*/
@Autowired
private MasterConfig masterConfig;
/**
* task response service
*/
@Autowired
private TaskEventService taskEventService;
/**
* storage operator
*/
@Autowired(required = false)
private StorageOperate storageOperate;
/**
* consumer thread pool
*/
private ThreadPoolExecutor consumerThreadPoolExecutor;
protected TaskPriorityQueueConsumer() {
super("TaskPriorityQueueConsumeThread");
}
@PostConstruct
public void init() {
this.consumerThreadPoolExecutor = (ThreadPoolExecutor) ThreadUtils
.newDaemonFixedThreadExecutor("TaskUpdateQueueConsumerThread", masterConfig.getDispatchTaskNumber());
log.info("Task priority queue consume thread staring");
super.start();
log.info("Task priority queue consume thread started");
}
@Override
public void run() {
int fetchTaskNum = masterConfig.getDispatchTaskNumber();
while (!ServerLifeCycleManager.isStopped()) {
try {
List<TaskPriority> failedDispatchTasks = this.batchDispatch(fetchTaskNum);
if (CollectionUtils.isNotEmpty(failedDispatchTasks)) {
log.info("{} tasks dispatch failed, will retry to dispatch", failedDispatchTasks.size());
TaskMetrics.incTaskDispatchFailed(failedDispatchTasks.size());
for (TaskPriority dispatchFailedTask : failedDispatchTasks) {
taskPriorityQueue.put(dispatchFailedTask);
}
// If the all task dispatch failed, will sleep for 1s to avoid the master cpu higher.
if (fetchTaskNum == failedDispatchTasks.size()) {
log.info("All tasks dispatch failed, will sleep a while to avoid the master cpu higher");
TimeUnit.MILLISECONDS.sleep(Constants.SLEEP_TIME_MILLIS);
}
}
} catch (Exception e) {
TaskMetrics.incTaskDispatchError();
log.error("dispatcher task error", e);
}
}
}
/**
* batch dispatch with thread pool
*/
public List<TaskPriority> batchDispatch(int fetchTaskNum) throws TaskPriorityQueueException, InterruptedException {
List<TaskPriority> failedDispatchTasks = Collections.synchronizedList(new ArrayList<>());
CountDownLatch latch = new CountDownLatch(fetchTaskNum);
for (int i = 0; i < fetchTaskNum; i++) {
TaskPriority taskPriority = taskPriorityQueue.poll(Constants.SLEEP_TIME_MILLIS, TimeUnit.MILLISECONDS);
if (Objects.isNull(taskPriority)) {
latch.countDown();
continue;
}
consumerThreadPoolExecutor.submit(() -> {
try {
try {
this.dispatchTask(taskPriority);
} catch (WorkerGroupNotFoundException e) {
// If the worker group not found, will not try to dispatch again.
// The task instance will be failed
// todo:
addDispatchFailedEvent(taskPriority);
} catch (ExecuteException e) {
failedDispatchTasks.add(taskPriority);
} catch (Exception e) {
log.error("Dispatch task error, meet an unknown exception", e);
failedDispatchTasks.add(taskPriority);
}
} finally {
// make sure the latch countDown
latch.countDown();
}
});
}
latch.await();
return failedDispatchTasks;
}
/**
* Dispatch task to worker.
*
* @param taskPriority taskPriority
* @return dispatch result, return true if dispatch success, return false if dispatch failed.
*/
protected void dispatchTask(TaskPriority taskPriority) throws ExecuteException {
TaskMetrics.incTaskDispatch();
WorkflowExecuteRunnable workflowExecuteRunnable =
processInstanceExecCacheManager.getByProcessInstanceId(taskPriority.getProcessInstanceId());
if (workflowExecuteRunnable == null) {
log.error("Cannot find the related processInstance of the task, taskPriority: {}", taskPriority);
return;
}
Optional<TaskInstance> taskInstanceOptional =
workflowExecuteRunnable.getTaskInstance(taskPriority.getTaskId());
if (!taskInstanceOptional.isPresent()) {
log.error("Cannot find the task instance from related processInstance, taskPriority: {}",
taskPriority);
// we return true, so that we will drop this task.
return;
}
TaskInstance taskInstance = taskInstanceOptional.get();
TaskExecutionContext context = taskPriority.getTaskExecutionContext();
ExecutionContext executionContext = ExecutionContext.builder()
.taskInstance(taskInstance)
.workerGroup(context.getWorkerGroup())
.executorType(ExecutorType.WORKER)
.message(toCommand(context))
.build();
if (isTaskNeedToCheck(taskPriority)) {
if (taskInstanceIsFinalState(taskPriority.getTaskId())) {
// when task finish, ignore this task, there is no need to dispatch anymore
log.info("Task {} is already finished, no need to dispatch, task instance id: {}",
taskInstance.getName(), taskInstance.getId());
return;
}
}
// check task is cache execution, and decide whether to dispatch
if (checkIsCacheExecution(taskInstance, context)) {
return;
}
dispatcher.dispatch(executionContext);
log.info("Master success dispatch task to worker, taskInstanceId: {}, worker: {}",
taskPriority.getTaskId(),
executionContext.getHost());
addDispatchEvent(context, executionContext);
}
/**
* add dispatch event
*/
private void addDispatchEvent(TaskExecutionContext context, ExecutionContext executionContext) {
TaskEvent taskEvent = TaskEvent.newDispatchEvent(context.getProcessInstanceId(), context.getTaskInstanceId(),
executionContext.getHost().getAddress());
taskEventService.addEvent(taskEvent);
}
private void addDispatchFailedEvent(TaskPriority taskPriority) {
TaskExecutionContext taskExecutionContext = taskPriority.getTaskExecutionContext();
TaskEvent taskEvent = TaskEvent.builder()
.processInstanceId(taskPriority.getProcessInstanceId())
.taskInstanceId(taskPriority.getTaskId())
.state(TaskExecutionStatus.FAILURE)
.logPath(taskExecutionContext.getLogPath())
.executePath(taskExecutionContext.getExecutePath())
.appIds(taskExecutionContext.getAppIds())
.processId(taskExecutionContext.getProcessId())
.varPool(taskExecutionContext.getVarPool())
.startTime(DateUtils.timeStampToDate(taskExecutionContext.getStartTime()))
.endTime(new Date())
.event(TaskEventType.RESULT)
.build();
taskEventService.addEvent(taskEvent);
}
private Message toCommand(TaskExecutionContext taskExecutionContext) {
// todo: we didn't set the host here, since right now we didn't need to retry this message.
TaskDispatchMessage requestCommand = new TaskDispatchMessage(taskExecutionContext,
masterConfig.getMasterAddress(),
taskExecutionContext.getHost(),
System.currentTimeMillis());
return requestCommand.convert2Command();
}
/**
* taskInstance is final state
* successfailurekillstoppausethreadwaiting is final state
*
* @param taskInstanceId taskInstanceId
* @return taskInstance is final state
*/
public boolean taskInstanceIsFinalState(int taskInstanceId) {
TaskInstance taskInstance = taskInstanceDao.findTaskInstanceById(taskInstanceId);
return taskInstance.getState().isFinished();
}
/**
* check if task need to check state, if true, refresh the checkpoint
*/
private boolean isTaskNeedToCheck(TaskPriority taskPriority) {
long now = System.currentTimeMillis();
if (now - taskPriority.getCheckpoint() > Constants.SECOND_TIME_MILLIS) {
taskPriority.setCheckpoint(now);
return true;
}
return false;
}
/**
* check if task is cache execution
* if the task is defined as cache execution, and we find the cache task instance is finished yet, we will not dispatch this task
* @param taskInstance taskInstance
* @param context context
* @return true if we will not dispatch this task, false if we will dispatch this task
*/
private boolean checkIsCacheExecution(TaskInstance taskInstance, TaskExecutionContext context) {
try {
// check if task is defined as a cache task
if (taskInstance.getIsCache().equals(Flag.NO)) {
return false;
}
// check if task is cache execution
String cacheKey = TaskCacheUtils.generateCacheKey(taskInstance, context, storageOperate);
TaskInstance cacheTaskInstance = taskInstanceDao.findTaskInstanceByCacheKey(cacheKey);
// if we can find the cache task instance, we will add cache event, and return true.
if (cacheTaskInstance != null) {
log.info("Task {} is cache, no need to dispatch, task instance id: {}",
taskInstance.getName(), taskInstance.getId());
addCacheEvent(taskInstance, cacheTaskInstance);
taskInstance.setCacheKey(TaskCacheUtils.generateTagCacheKey(cacheTaskInstance.getId(), cacheKey));
return true;
} else {
// if we can not find cache task, update cache key, and return false. the task will be dispatched
taskInstance.setCacheKey(TaskCacheUtils.generateTagCacheKey(taskInstance.getId(), cacheKey));
}
} catch (Exception e) {
log.error("checkIsCacheExecution error", e);
}
return false;
}
private void addCacheEvent(TaskInstance taskInstance, TaskInstance cacheTaskInstance) {
if (cacheTaskInstance == null) {
return;
}
TaskEvent taskEvent = TaskEvent.newCacheEvent(taskInstance.getProcessInstanceId(), taskInstance.getId(),
cacheTaskInstance.getId());
taskEventService.addEvent(taskEvent);
}
}

118
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/ExecutorDispatcher.java

@ -1,118 +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.server.master.dispatch;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException;
import org.apache.dolphinscheduler.server.master.dispatch.executor.ExecutorManager;
import org.apache.dolphinscheduler.server.master.dispatch.executor.NettyExecutorManager;
import org.apache.dolphinscheduler.server.master.dispatch.host.HostManager;
import org.apache.commons.lang3.StringUtils;
import java.util.concurrent.ConcurrentHashMap;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.InitializingBean;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
/**
* executor dispatcher
*/
@Service
@Slf4j
public class ExecutorDispatcher implements InitializingBean {
/**
* netty executor manager
*/
@Autowired
private NettyExecutorManager nettyExecutorManager;
/**
* round robin host manager
*/
@Autowired
private HostManager hostManager;
/**
* executor manager
*/
private final ConcurrentHashMap<ExecutorType, ExecutorManager<Boolean>> executorManagers;
/**
* constructor
*/
public ExecutorDispatcher() {
this.executorManagers = new ConcurrentHashMap<>();
}
/**
* task dispatch
*
* @param context context
* @return result
* @throws ExecuteException if error throws ExecuteException
*/
public void dispatch(final ExecutionContext context) throws ExecuteException {
// get executor manager
ExecutorManager<Boolean> executorManager = this.executorManagers.get(context.getExecutorType());
if (executorManager == null) {
throw new ExecuteException("no ExecutorManager for type : " + context.getExecutorType());
}
// host select
Host host = hostManager.select(context);
if (StringUtils.isEmpty(host.getAddress())) {
log.warn("fail to execute : {} due to no suitable worker, current task needs worker group {} to execute",
context.getMessage(), context.getWorkerGroup());
throw new ExecuteException("no suitable worker");
}
context.setHost(host);
executorManager.beforeExecute(context);
try {
// task execute
executorManager.execute(context);
} finally {
executorManager.afterExecute(context);
}
}
/**
* register init
* @throws Exception if error throws Exception
*/
@Override
public void afterPropertiesSet() throws Exception {
register(ExecutorType.WORKER, nettyExecutorManager);
register(ExecutorType.CLIENT, nettyExecutorManager);
}
/**
* register
* @param type executor type
* @param executorManager executorManager
*/
public void register(ExecutorType type, ExecutorManager executorManager) {
executorManagers.put(type, executorManager);
}
}

8
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/enums/ExecutorType.java

@ -17,12 +17,12 @@
package org.apache.dolphinscheduler.server.master.dispatch.enums;
/**
* executor type
*/
// todo: refactor this enum
public enum ExecutorType {
WORKER,
CLIENT;
CLIENT,
MASTER,
;
}

30
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/CommonHostManager.java

@ -18,8 +18,6 @@
package org.apache.dolphinscheduler.server.master.dispatch.host;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker;
import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager;
@ -29,6 +27,7 @@ import org.apache.commons.collections4.CollectionUtils;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import org.springframework.beans.factory.annotation.Autowired;
@ -44,32 +43,13 @@ public abstract class CommonHostManager implements HostManager {
@Autowired
protected ServerNodeManager serverNodeManager;
/**
* select host
*
* @param context context
* @return host
* @throws WorkerGroupNotFoundException If the worker group not found
*/
@Override
public Host select(ExecutionContext context) throws WorkerGroupNotFoundException {
List<HostWorker> candidates = null;
String workerGroup = context.getWorkerGroup();
ExecutorType executorType = context.getExecutorType();
switch (executorType) {
case WORKER:
candidates = getWorkerCandidates(workerGroup);
break;
case CLIENT:
break;
default:
throw new IllegalArgumentException("invalid executorType : " + executorType);
}
public Optional<Host> select(String workerGroup) throws WorkerGroupNotFoundException {
List<HostWorker> candidates = getWorkerCandidates(workerGroup);
if (CollectionUtils.isEmpty(candidates)) {
return new Host();
return Optional.empty();
}
return select(candidates);
return Optional.ofNullable(select(candidates));
}
protected abstract HostWorker select(Collection<HostWorker> nodes);

12
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/HostManager.java

@ -18,21 +18,15 @@
package org.apache.dolphinscheduler.server.master.dispatch.host;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
import java.util.Optional;
/**
* host manager
*/
public interface HostManager {
/**
* select host
*
* @param context context
* @return host
* @throws WorkerGroupNotFoundException If the worker group does exist
*/
Host select(ExecutionContext context) throws WorkerGroupNotFoundException;
Optional<Host> select(String workerGroup) throws WorkerGroupNotFoundException;
}

16
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/dispatch/host/LowerWeightHostManager.java

@ -20,7 +20,6 @@ package org.apache.dolphinscheduler.server.master.dispatch.host;
import org.apache.dolphinscheduler.common.constants.Constants;
import org.apache.dolphinscheduler.common.model.WorkerHeartBeat;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWeight;
import org.apache.dolphinscheduler.server.master.dispatch.host.assign.HostWorker;
@ -70,20 +69,13 @@ public class LowerWeightHostManager extends CommonHostManager {
serverNodeManager.addWorkerInfoChangeListener(new WorkerWeightListener());
}
/**
* select host
*
* @param context context
* @return host
* @throws WorkerGroupNotFoundException If the worker group not found
*/
@Override
public Host select(ExecutionContext context) throws WorkerGroupNotFoundException {
Set<HostWeight> workerHostWeights = getWorkerHostWeights(context.getWorkerGroup());
public Optional<Host> select(String workerGroup) throws WorkerGroupNotFoundException {
Set<HostWeight> workerHostWeights = getWorkerHostWeights(workerGroup);
if (CollectionUtils.isNotEmpty(workerHostWeights)) {
return selector.select(workerHostWeights).getHost();
return Optional.ofNullable(selector.select(workerHostWeights).getHost());
}
return new Host();
return Optional.empty();
}
@Override

84
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskRejectByWorkerEventHandler.java

@ -1,84 +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.server.master.event;
import org.apache.dolphinscheduler.common.enums.TaskEventType;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.remote.command.task.TaskRejectMessageAck;
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Component
public class TaskRejectByWorkerEventHandler implements TaskEventHandler {
@Autowired
private ProcessInstanceExecCacheManager processInstanceExecCacheManager;
@Autowired
private MasterConfig masterConfig;
@Override
public void handleTaskEvent(TaskEvent taskEvent) throws TaskEventHandleError {
int taskInstanceId = taskEvent.getTaskInstanceId();
int processInstanceId = taskEvent.getProcessInstanceId();
WorkflowExecuteRunnable workflowExecuteRunnable = this.processInstanceExecCacheManager.getByProcessInstanceId(
processInstanceId);
if (workflowExecuteRunnable == null) {
sendAckToWorker(taskEvent);
throw new TaskEventHandleError(
"Handle task reject event error, cannot find related workflow instance from cache, will discard this event");
}
TaskInstance taskInstance = workflowExecuteRunnable.getTaskInstance(taskInstanceId).orElseThrow(() -> {
sendAckToWorker(taskEvent);
return new TaskEventHandleError(
"Handle task reject event error, cannot find the taskInstance from cache, will discord this event");
});
try {
// todo: If the worker submit multiple reject response to master, the task instance may be dispatch
// multiple,
// we need to control the worker overload by master rather than worker
// if the task resubmit and the worker failover, this task may be dispatch twice?
// todo: we need to clear the taskInstance host and rollback the status to submit.
workflowExecuteRunnable.resubmit(taskInstance.getTaskCode());
sendAckToWorker(taskEvent);
} catch (Exception ex) {
throw new TaskEventHandleError("Handle task reject event error", ex);
}
}
public void sendAckToWorker(TaskEvent taskEvent) {
TaskRejectMessageAck taskRejectMessageAck = new TaskRejectMessageAck(true,
taskEvent.getTaskInstanceId(),
masterConfig.getMasterAddress(),
taskEvent.getWorkerAddress(),
System.currentTimeMillis());
taskEvent.getChannel().writeAndFlush(taskRejectMessageAck.convert2Command());
}
@Override
public TaskEventType getHandleEventType() {
return TaskEventType.WORKER_REJECT;
}
}

24
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskStateEventHandler.java

@ -22,8 +22,6 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.remote.exceptions.RemotingException;
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
import org.apache.dolphinscheduler.server.master.runner.task.ITaskProcessor;
import org.apache.dolphinscheduler.server.master.runner.task.TaskAction;
import java.util.Map;
import java.util.Optional;
@ -55,15 +53,15 @@ public class TaskStateEventHandler implements StateEventHandler {
log.info(
"Handle task instance state event, the current task instance state {} will be changed to {}",
task.getState(), taskStateEvent.getStatus());
task.getState().name(), taskStateEvent.getStatus().name());
Map<Long, Integer> completeTaskMap = workflowExecuteRunnable.getCompleteTaskMap();
if (task.getState().isFinished()
&& (taskStateEvent.getStatus() != null && taskStateEvent.getStatus().isRunning())) {
String errorMessage = String.format(
"The current task instance state is %s, but the task state event status is %s, so the task state event will be ignored",
task.getState(),
taskStateEvent.getStatus());
task.getState().name(),
taskStateEvent.getStatus().name());
log.warn(errorMessage);
throw new StateEventHandleError(errorMessage);
}
@ -85,21 +83,7 @@ public class TaskStateEventHandler implements StateEventHandler {
}
return true;
}
Map<Long, ITaskProcessor> activeTaskProcessMap = workflowExecuteRunnable.getActiveTaskProcessMap();
if (activeTaskProcessMap.containsKey(task.getTaskCode())) {
ITaskProcessor iTaskProcessor = activeTaskProcessMap.get(task.getTaskCode());
iTaskProcessor.action(TaskAction.RUN);
if (iTaskProcessor.taskInstance().getState().isFinished()) {
if (iTaskProcessor.taskInstance().getState() != task.getState()) {
task.setState(iTaskProcessor.taskInstance().getState());
}
workflowExecuteRunnable.taskFinished(task);
}
return true;
}
throw new StateEventHandleError(
"Task state event handle error, due to the task is not in activeTaskProcessorMaps");
return true;
}
@Override

12
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskTimeoutStateEventHandler.java

@ -23,8 +23,7 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy;
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
import org.apache.dolphinscheduler.server.master.runner.task.ITaskProcessor;
import org.apache.dolphinscheduler.server.master.runner.task.TaskAction;
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable;
import java.util.Map;
@ -57,13 +56,12 @@ public class TaskTimeoutStateEventHandler implements StateEventHandler {
}
TaskTimeoutStrategy taskTimeoutStrategy = taskInstance.getTaskDefine()
.getTimeoutNotifyStrategy();
Map<Long, ITaskProcessor> activeTaskProcessMap = workflowExecuteRunnable
.getActiveTaskProcessMap();
Map<Long, DefaultTaskExecuteRunnable> taskExecuteRunnableMap =
workflowExecuteRunnable.getTaskExecuteRunnableMap();
if ((TaskTimeoutStrategy.FAILED == taskTimeoutStrategy
|| TaskTimeoutStrategy.WARNFAILED == taskTimeoutStrategy)) {
if (activeTaskProcessMap.containsKey(taskInstance.getTaskCode())) {
ITaskProcessor taskProcessor = activeTaskProcessMap.get(taskInstance.getTaskCode());
taskProcessor.action(TaskAction.TIMEOUT);
if (taskExecuteRunnableMap.containsKey(taskInstance.getTaskCode())) {
taskExecuteRunnableMap.get(taskInstance.getTaskCode()).timeout();
} else {
log.warn(
"cannot find the task processor for task {}, so skip task processor action.",

8
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/TaskUpdatePidEventHandler.java

@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.common.enums.TaskEventType;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.dao.utils.TaskInstanceUtils;
import org.apache.dolphinscheduler.remote.command.task.TaskUpdatePidAckMessage;
import org.apache.dolphinscheduler.remote.command.task.TaskUpdateRuntimeAckMessage;
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
@ -91,9 +91,9 @@ public class TaskUpdatePidEventHandler implements TaskEventHandler {
private void sendAckToWorker(TaskEvent taskEvent) {
// If event handle success, send ack to worker to otherwise the worker will retry this event
TaskUpdatePidAckMessage taskUpdatePidAckMessage =
new TaskUpdatePidAckMessage(true, taskEvent.getTaskInstanceId());
taskEvent.getChannel().writeAndFlush(taskUpdatePidAckMessage.convert2Command());
TaskUpdateRuntimeAckMessage taskUpdateRuntimeAckMessage =
new TaskUpdateRuntimeAckMessage(true, taskEvent.getTaskInstanceId());
taskEvent.getChannel().writeAndFlush(taskUpdateRuntimeAckMessage.convert2Command());
}
@Override

3
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/event/WorkflowStartEventHandler.java

@ -47,9 +47,6 @@ public class WorkflowStartEventHandler implements WorkflowEventHandler {
@Autowired
private WorkflowExecuteThreadPool workflowExecuteThreadPool;
@Autowired
private WorkflowEventQueue workflowEventQueue;
@Override
public void handleWorkflowEvent(final WorkflowEvent workflowEvent) throws WorkflowEventHandleError {
log.info("Handle workflow start event, begin to start a workflow, event: {}", workflowEvent);

26
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/LogicTaskFactoryNotFoundException.java

@ -0,0 +1,26 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.exception;
public class LogicTaskFactoryNotFoundException extends MasterException {
public LogicTaskFactoryNotFoundException(String message) {
super(message);
}
}

30
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/LogicTaskInitializeException.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.exception;
public class LogicTaskInitializeException extends MasterException {
public LogicTaskInitializeException(String message) {
super(message);
}
public LogicTaskInitializeException(String message, Throwable cause) {
super(message, cause);
}
}

29
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/MasterTaskExecuteException.java

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.exception;
public class MasterTaskExecuteException extends MasterException {
public MasterTaskExecuteException(String message) {
super(message);
}
public MasterTaskExecuteException(String message, Throwable cause) {
super(message, cause);
}
}

29
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskDispatchException.java

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.exception;
public class TaskDispatchException extends MasterException {
public TaskDispatchException(String message) {
super(message);
}
public TaskDispatchException(String message, Throwable cause) {
super(message, cause);
}
}

30
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecuteRunnableCreateException.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.exception;
public class TaskExecuteRunnableCreateException extends MasterException {
public TaskExecuteRunnableCreateException(String message) {
super(message);
}
public TaskExecuteRunnableCreateException(String message, Throwable throwable) {
super(message, throwable);
}
}

26
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/TaskExecutionContextCreateException.java

@ -0,0 +1,26 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.exception;
public class TaskExecutionContextCreateException extends MasterException {
public TaskExecutionContextCreateException(String message) {
super(message);
}
}

29
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/exception/WorkflowCreateException.java

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.exception;
public class WorkflowCreateException extends MasterException {
public WorkflowCreateException(String message) {
super(message);
}
public WorkflowCreateException(String message, Throwable throwable) {
super(message, throwable);
}
}

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/CacheProcessor.java

@ -22,7 +22,7 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.cache.CacheExpireRequest;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import lombok.extern.slf4j.Slf4j;
@ -38,7 +38,7 @@ import io.netty.channel.Channel;
*/
@Component
@Slf4j
public class CacheProcessor implements NettyRequestProcessor {
public class CacheProcessor implements MasterRpcProcessor {
@Autowired
private CacheManager cacheManager;

130
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskDispatchProcessor.java

@ -0,0 +1,130 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.processor;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskDispatchRequest;
import org.apache.dolphinscheduler.remote.command.task.TaskDispatchResponse;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.runner.MasterDelayTaskExecuteRunnableDelayQueue;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterDelayTaskExecuteRunnable;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnableFactoryBuilder;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutionContextHolder;
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager;
import java.util.concurrent.TimeUnit;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import io.netty.channel.Channel;
@Slf4j
@Component
public class MasterTaskDispatchProcessor implements MasterRpcProcessor {
@Autowired
private MasterTaskExecuteRunnableFactoryBuilder masterTaskExecuteRunnableFactoryBuilder;
@Autowired
private MasterMessageSenderManager masterMessageSenderManager;
@Autowired
private MasterDelayTaskExecuteRunnableDelayQueue masterDelayTaskExecuteRunnableDelayQueue;
@Override
public void process(Channel channel, Message message) {
TaskDispatchRequest taskDispatchRequest = JSONUtils.parseObject(message.getBody(), TaskDispatchRequest.class);
log.info("Receive task dispatch request, command: {}", taskDispatchRequest);
TaskExecutionContext taskExecutionContext = taskDispatchRequest.getTaskExecutionContext();
taskExecutionContext.setLogPath(LogUtils.getTaskInstanceLogFullPath(taskExecutionContext));
try (
final LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC(
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId());
final LogUtils.MDCAutoClosableContext mdcAutoClosableContext1 =
LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath())) {
MasterTaskExecutionContextHolder.putTaskExecutionContext(taskExecutionContext);
// todo: calculate the delay in master dispatcher then we don't need to use a queue to store the task
long remainTime =
DateUtils.getRemainTime(DateUtils.timeStampToDate(taskExecutionContext.getFirstSubmitTime()),
TimeUnit.SECONDS.toMillis(taskExecutionContext.getDelayTime()));
// todo: move this to the master delay queue
if (remainTime > 0) {
log.info("Current taskInstance: {} is choose delay execution, delay time: {}ms, remainTime: {}ms",
taskExecutionContext.getTaskName(),
TimeUnit.SECONDS.toMillis(taskExecutionContext.getDelayTime()), remainTime);
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.DELAY_EXECUTION);
masterMessageSenderManager.getMasterTaskExecuteResultMessageSender().sendMessage(taskExecutionContext);
}
MasterDelayTaskExecuteRunnable masterDelayTaskExecuteRunnable = masterTaskExecuteRunnableFactoryBuilder
.createWorkerDelayTaskExecuteRunnableFactory(taskExecutionContext.getTaskType())
.createWorkerTaskExecuteRunnable(taskExecutionContext);
if (masterDelayTaskExecuteRunnableDelayQueue
.submitMasterDelayTaskExecuteRunnable(masterDelayTaskExecuteRunnable)) {
log.info(
"Submit task: {} to MasterDelayTaskExecuteRunnableDelayQueue success",
taskExecutionContext.getTaskName());
sendDispatchSuccessResult(channel, message, taskExecutionContext);
} else {
log.error(
"Submit task: {} to MasterDelayTaskExecuteRunnableDelayQueue failed, current task waiting queue size: {} is full",
taskExecutionContext.getTaskName(), masterDelayTaskExecuteRunnableDelayQueue.size());
sendDispatchRejectResult(channel, message, taskExecutionContext);
}
} catch (Exception ex) {
log.error("Handle task dispatch request error, command: {}", taskDispatchRequest, ex);
sendDispatchFailedResult(channel, message, taskExecutionContext, ex);
}
}
private void sendDispatchSuccessResult(Channel channel, Message dispatchRequest,
TaskExecutionContext taskExecutionContext) {
TaskDispatchResponse taskDispatchResponse =
TaskDispatchResponse.success(taskExecutionContext.getTaskInstanceId());
channel.writeAndFlush(taskDispatchResponse.convert2Command(dispatchRequest.getOpaque()));
}
private void sendDispatchRejectResult(Channel channel, Message dispatchRequest,
TaskExecutionContext taskExecutionContext) {
TaskDispatchResponse taskDispatchResponse =
TaskDispatchResponse.failed(taskExecutionContext.getTaskInstanceId(), "Task dispatch queue is full");
channel.writeAndFlush(taskDispatchResponse.convert2Command(dispatchRequest.getOpaque()));
}
private void sendDispatchFailedResult(Channel channel, Message dispatchRequest,
TaskExecutionContext taskExecutionContext, Throwable throwable) {
TaskDispatchResponse taskDispatchResponse =
TaskDispatchResponse.failed(taskExecutionContext.getTaskInstanceId(), throwable.getMessage());
channel.writeAndFlush(taskDispatchResponse.convert2Command(dispatchRequest.getOpaque()));
}
@Override
public MessageType getCommandType() {
return MessageType.TASK_DISPATCH_REQUEST;
}
}

83
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskKillProcessor.java

@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.processor;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskKillRequest;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.MasterDelayTaskExecuteRunnableDelayQueue;
import org.apache.dolphinscheduler.server.master.runner.MasterTaskExecuteRunnableThreadPool;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecutionContextHolder;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import io.netty.channel.Channel;
@Slf4j
@Component
public class MasterTaskKillProcessor implements MasterRpcProcessor {
@Autowired
private MasterTaskExecuteRunnableThreadPool masterTaskExecuteRunnableThreadPool;
@Autowired
private MasterDelayTaskExecuteRunnableDelayQueue masterDelayTaskExecuteRunnableDelayQueue;
@Override
public void process(Channel channel, Message message) {
TaskKillRequest taskKillRequest = JSONUtils.parseObject(message.getBody(), TaskKillRequest.class);
log.info("Master receive task kill request: {}", taskKillRequest);
int taskInstanceId = taskKillRequest.getTaskInstanceId();
try (LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setTaskInstanceIdMDC(taskInstanceId)) {
TaskExecutionContext taskExecutionContext =
MasterTaskExecutionContextHolder.getTaskExecutionContext(taskInstanceId);
if (taskExecutionContext == null) {
log.error("Cannot find the TaskExecutionContext, this task may already been killed");
return;
}
MasterTaskExecuteRunnable masterTaskExecuteRunnable =
masterTaskExecuteRunnableThreadPool.getMasterTaskExecuteRunnable(taskInstanceId);
if (masterTaskExecuteRunnable == null) {
log.error("Cannot find the MasterTaskExecuteRunnable, this task may already been killed");
return;
}
try {
masterTaskExecuteRunnable.cancelTask();
masterDelayTaskExecuteRunnableDelayQueue
.removeMasterDelayTaskExecuteRunnable(masterTaskExecuteRunnable);
} catch (MasterTaskExecuteException e) {
log.error("Cancel MasterTaskExecuteRunnable failed ", e);
}
}
}
@Override
public MessageType getCommandType() {
return MessageType.TASK_KILL_REQUEST;
}
}

68
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/MasterTaskPauseProcessor.java

@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.processor;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskPauseRequest;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.MasterTaskExecuteRunnableThreadPool;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import io.netty.channel.Channel;
@Slf4j
@Component
public class MasterTaskPauseProcessor implements MasterRpcProcessor {
@Autowired
private MasterTaskExecuteRunnableThreadPool masterTaskExecuteRunnableThreadPool;
@Override
public void process(Channel channel, Message message) {
TaskPauseRequest taskPauseRequest = JSONUtils.parseObject(message.getBody(), TaskPauseRequest.class);
MasterTaskExecuteRunnable masterTaskExecuteRunnable =
masterTaskExecuteRunnableThreadPool.getMasterTaskExecuteRunnable(taskPauseRequest.getTaskInstanceId());
if (masterTaskExecuteRunnable == null) {
log.info("Cannot find the MasterTaskExecuteRunnable");
return;
}
TaskExecutionContext taskExecutionContext = masterTaskExecuteRunnable.getTaskExecutionContext();
try (
LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC(
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId())) {
masterTaskExecuteRunnable.pauseTask();
} catch (MasterTaskExecuteException e) {
log.error("Pause MasterTaskExecuteRunnable failed", e);
}
}
@Override
public MessageType getCommandType() {
return MessageType.PAUSE_TASK_INSTANCE;
}
}

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/StateEventProcessor.java

@ -24,7 +24,7 @@ import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.workflow.WorkflowStateEventChangeRequest;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.event.StateEvent;
import org.apache.dolphinscheduler.server.master.event.TaskStateEvent;
import org.apache.dolphinscheduler.server.master.event.WorkflowStateEvent;
@ -42,7 +42,7 @@ import io.netty.channel.Channel;
*/
@Component
@Slf4j
public class StateEventProcessor implements NettyRequestProcessor {
public class StateEventProcessor implements MasterRpcProcessor {
@Autowired
private StateEventResponseService stateEventResponseService;

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteResultProcessor.java

@ -22,7 +22,7 @@ import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskExecuteResultMessage;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
@ -38,7 +38,7 @@ import io.netty.channel.Channel;
*/
@Component
@Slf4j
public class TaskExecuteResultProcessor implements NettyRequestProcessor {
public class TaskExecuteResultProcessor implements MasterRpcProcessor {
@Autowired
private TaskEventService taskEventService;

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteRunningProcessor.java

@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskExecuteRunningMessage;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
@ -37,7 +37,7 @@ import io.netty.channel.Channel;
*/
@Component
@Slf4j
public class TaskExecuteRunningProcessor implements NettyRequestProcessor {
public class TaskExecuteRunningProcessor implements MasterRpcProcessor {
@Autowired
private TaskEventService taskEventService;

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskExecuteStartProcessor.java

@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.dao.repository.TaskDefinitionDao;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskExecuteStartMessage;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.runner.StreamTaskExecuteRunnable;
import org.apache.dolphinscheduler.server.master.runner.StreamTaskExecuteThreadPool;
@ -39,7 +39,7 @@ import io.netty.channel.Channel;
*/
@Component
@Slf4j
public class TaskExecuteStartProcessor implements NettyRequestProcessor {
public class TaskExecuteStartProcessor implements MasterRpcProcessor {
@Autowired
private StreamTaskExecuteThreadPool streamTaskExecuteThreadPool;

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskForceStartProcessor.java

@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskForceStartRequest;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.event.TaskStateEvent;
import org.apache.dolphinscheduler.server.master.processor.queue.StateEventResponseService;
@ -36,7 +36,7 @@ import io.netty.channel.Channel;
@Component
@Slf4j
public class TaskForceStartProcessor implements NettyRequestProcessor {
public class TaskForceStartProcessor implements MasterRpcProcessor {
@Autowired
private StateEventResponseService stateEventResponseService;

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskKillResponseProcessor.java

@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskKillResponse;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import lombok.extern.slf4j.Slf4j;
@ -34,7 +34,7 @@ import io.netty.channel.Channel;
*/
@Component
@Slf4j
public class TaskKillResponseProcessor implements NettyRequestProcessor {
public class TaskKillResponseProcessor implements MasterRpcProcessor {
/**
* task final result response

68
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskRecallProcessor.java

@ -1,68 +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.server.master.processor;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskRejectMessage;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import io.netty.channel.Channel;
/**
* task recall processor
*/
@Component
@Slf4j
public class TaskRecallProcessor implements NettyRequestProcessor {
@Autowired
private TaskEventService taskEventService;
/**
* task ack process
*
* @param channel channel channel
* @param message command TaskExecuteAckCommand
*/
@Override
public void process(Channel channel, Message message) {
TaskRejectMessage recallCommand = JSONUtils.parseObject(message.getBody(), TaskRejectMessage.class);
TaskEvent taskEvent = TaskEvent.newRecallEvent(recallCommand, channel);
try (
final LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC(
recallCommand.getProcessInstanceId(), recallCommand.getTaskInstanceId())) {
log.info("Receive task recall command: {}", recallCommand);
taskEventService.addEvent(taskEvent);
}
}
@Override
public MessageType getCommandType() {
return MessageType.TASK_REJECT;
}
}

12
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskUpdatePidProcessor.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskUpdateRuntimeProcessor.java

@ -20,8 +20,8 @@ package org.apache.dolphinscheduler.server.master.processor;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskUpdatePidMessage;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.command.task.TaskUpdateRuntimeMessage;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
@ -37,7 +37,7 @@ import io.netty.channel.Channel;
*/
@Component
@Slf4j
public class TaskUpdatePidProcessor implements NettyRequestProcessor {
public class TaskUpdateRuntimeProcessor implements MasterRpcProcessor {
@Autowired
private TaskEventService taskEventService;
@ -50,8 +50,8 @@ public class TaskUpdatePidProcessor implements NettyRequestProcessor {
*/
@Override
public void process(Channel channel, Message message) {
TaskUpdatePidMessage taskUpdatePidRequest =
JSONUtils.parseObject(message.getBody(), TaskUpdatePidMessage.class);
TaskUpdateRuntimeMessage taskUpdatePidRequest =
JSONUtils.parseObject(message.getBody(), TaskUpdateRuntimeMessage.class);
log.info("taskUpdatePidCommand: {}", taskUpdatePidRequest);
TaskEvent taskEvent = TaskEvent.newUpdatePidEvent(taskUpdatePidRequest,
@ -62,7 +62,7 @@ public class TaskUpdatePidProcessor implements NettyRequestProcessor {
@Override
public MessageType getCommandType() {
return MessageType.TASK_UPDATE_PID_MESSAGE;
return MessageType.TASK_UPDATE_RUNTIME_MESSAGE;
}
}

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/TaskWakeupProcessor.java

@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskWakeupRequest;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.event.TaskStateEvent;
import org.apache.dolphinscheduler.server.master.processor.queue.StateEventResponseService;
@ -39,7 +39,7 @@ import io.netty.channel.Channel;
*/
@Component
@Slf4j
public class TaskWakeupProcessor implements NettyRequestProcessor {
public class TaskWakeupProcessor implements MasterRpcProcessor {
@Autowired
private StateEventResponseService stateEventResponseService;

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowExecutingDataRequestProcessor.java

@ -23,7 +23,7 @@ import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.workflow.WorkflowExecutingDataRequest;
import org.apache.dolphinscheduler.remote.command.workflow.WorkflowExecutingDataResponse;
import org.apache.dolphinscheduler.remote.dto.WorkflowExecuteDto;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.service.ExecutingService;
import java.util.Optional;
@ -40,7 +40,7 @@ import io.netty.channel.Channel;
*/
@Component
@Slf4j
public class WorkflowExecutingDataRequestProcessor implements NettyRequestProcessor {
public class WorkflowExecutingDataRequestProcessor implements MasterRpcProcessor {
@Autowired
private ExecutingService executingService;

4
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/WorkflowMetricsCleanUpProcessor.java

@ -21,7 +21,7 @@ import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.workflow.WorkflowMetricsCleanUpRequest;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics;
import org.springframework.stereotype.Component;
@ -29,7 +29,7 @@ import org.springframework.stereotype.Component;
import io.netty.channel.Channel;
@Component
public class WorkflowMetricsCleanUpProcessor implements NettyRequestProcessor {
public class WorkflowMetricsCleanUpProcessor implements MasterRpcProcessor {
@Override
public void process(Channel channel, Message message) {

2
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/StateEventResponseService.java

@ -114,7 +114,7 @@ public class StateEventResponseService {
try {
stateEvent = eventQueue.take();
} catch (InterruptedException e) {
log.warn("State event loop service interrupted, will stop this loop", e);
log.warn("State event loop service interrupted, will stop loop");
Thread.currentThread().interrupt();
break;
}

14
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskEvent.java

@ -22,8 +22,7 @@ import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.remote.command.task.TaskExecuteResultMessage;
import org.apache.dolphinscheduler.remote.command.task.TaskExecuteRunningMessage;
import org.apache.dolphinscheduler.remote.command.task.TaskRejectMessage;
import org.apache.dolphinscheduler.remote.command.task.TaskUpdatePidMessage;
import org.apache.dolphinscheduler.remote.command.task.TaskUpdateRuntimeMessage;
import java.util.Date;
@ -145,15 +144,6 @@ public class TaskEvent {
return event;
}
public static TaskEvent newRecallEvent(TaskRejectMessage command, Channel channel) {
TaskEvent event = new TaskEvent();
event.setTaskInstanceId(command.getTaskInstanceId());
event.setProcessInstanceId(command.getProcessInstanceId());
event.setChannel(channel);
event.setEvent(TaskEventType.WORKER_REJECT);
return event;
}
public static TaskEvent newCacheEvent(int processInstanceId, int taskInstanceId, int cacheTaskInstanceId) {
TaskEvent event = new TaskEvent();
event.setProcessInstanceId(processInstanceId);
@ -163,7 +153,7 @@ public class TaskEvent {
return event;
}
public static TaskEvent newUpdatePidEvent(TaskUpdatePidMessage command, Channel channel, String workerAddress) {
public static TaskEvent newUpdatePidEvent(TaskUpdateRuntimeMessage command, Channel channel, String workerAddress) {
TaskEvent event = new TaskEvent();
event.setProcessInstanceId(command.getProcessInstanceId());
event.setTaskInstanceId(command.getTaskInstanceId());

10
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRPCServer.java

@ -19,7 +19,7 @@ package org.apache.dolphinscheduler.server.master.rpc;
import org.apache.dolphinscheduler.remote.NettyRemotingServer;
import org.apache.dolphinscheduler.remote.config.NettyServerConfig;
import org.apache.dolphinscheduler.remote.processor.NettyRequestProcessor;
import org.apache.dolphinscheduler.remote.processor.MasterRpcProcessor;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import java.util.List;
@ -42,7 +42,7 @@ public class MasterRPCServer implements AutoCloseable {
private MasterConfig masterConfig;
@Autowired
private List<NettyRequestProcessor> nettyRequestProcessors;
private List<MasterRpcProcessor> masterRpcProcessors;
public void start() {
log.info("Starting Master RPC Server...");
@ -50,9 +50,9 @@ public class MasterRPCServer implements AutoCloseable {
NettyServerConfig serverConfig = new NettyServerConfig();
serverConfig.setListenPort(masterConfig.getListenPort());
this.nettyRemotingServer = new NettyRemotingServer(serverConfig);
for (NettyRequestProcessor nettyRequestProcessor : nettyRequestProcessors) {
this.nettyRemotingServer.registerProcessor(nettyRequestProcessor);
log.info("Success register netty processor: {}", nettyRequestProcessor.getClass().getName());
for (MasterRpcProcessor masterRpcProcessor : masterRpcProcessors) {
this.nettyRemotingServer.registerProcessor(masterRpcProcessor);
log.info("Success register netty processor: {}", masterRpcProcessor.getClass().getName());
}
this.nettyRemotingServer.start();
log.info("Started Master RPC Server...");

3
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/rpc/MasterRpcClient.java

@ -46,4 +46,7 @@ public class MasterRpcClient {
return client.sendSync(host, rpcMessage, DEFAULT_TIME_OUT_MILLS);
}
public void send(Host of, Message message) throws RemotingException {
client.send(of, message);
}
}

132
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/BaseTaskDispatcher.java

@ -0,0 +1,132 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner;
import static com.google.common.base.Preconditions.checkNotNull;
import org.apache.dolphinscheduler.common.enums.TaskEventType;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.task.TaskDispatchRequest;
import org.apache.dolphinscheduler.remote.command.task.TaskDispatchResponse;
import org.apache.dolphinscheduler.remote.exceptions.RemotingException;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient;
import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatcher;
import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable;
import java.util.Date;
import java.util.Optional;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public abstract class BaseTaskDispatcher implements TaskDispatcher {
protected final TaskEventService taskEventService;
protected final MasterConfig masterConfig;
protected final MasterRpcClient masterRpcClient;
protected BaseTaskDispatcher(TaskEventService taskEventService,
MasterConfig masterConfig,
MasterRpcClient masterRpcClient) {
this.taskEventService = checkNotNull(taskEventService);
this.masterConfig = checkNotNull(masterConfig);
this.masterRpcClient = checkNotNull(masterRpcClient);
}
@Override
public void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException {
Host taskInstanceDispatchHost;
try {
taskInstanceDispatchHost = getTaskInstanceDispatchHost(taskExecuteRunnable)
.orElseThrow(() -> new TaskDispatchException("Cannot find the host to execute task."));
} catch (WorkerGroupNotFoundException workerGroupNotFoundException) {
log.error("Dispatch task: {} failed, worker group not found.",
taskExecuteRunnable.getTaskExecutionContext().getTaskName(), workerGroupNotFoundException);
addDispatchFailedEvent(taskExecuteRunnable);
return;
}
taskExecuteRunnable.getTaskExecutionContext().setHost(taskInstanceDispatchHost.getAddress());
doDispatch(taskExecuteRunnable);
taskExecuteRunnable.getTaskInstance().setHost(taskInstanceDispatchHost.getAddress());
log.info("Success dispatch task {} to {}.", taskExecuteRunnable.getTaskExecutionContext().getTaskName(),
taskInstanceDispatchHost.getAddress());
addDispatchEvent(taskExecuteRunnable);
}
protected abstract Optional<Host> getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) throws TaskDispatchException, WorkerGroupNotFoundException;
protected void doDispatch(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException {
TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext();
TaskDispatchRequest taskDispatchRequest = new TaskDispatchRequest(taskExecutionContext);
try {
Message message = masterRpcClient.sendSyncCommand(Host.of(taskExecutionContext.getHost()),
taskDispatchRequest.convert2Command());
TaskDispatchResponse taskDispatchResponse =
JSONUtils.parseObject(message.getBody(), TaskDispatchResponse.class);
if (!taskDispatchResponse.isDispatchSuccess()) {
throw new TaskDispatchException(String.format("Dispatch task to %s failed, response is: %s",
taskExecutionContext.getHost(), taskDispatchResponse));
}
} catch (InterruptedException e) {
// This exception should only happen when we close the server.
Thread.currentThread().interrupt();
throw new TaskDispatchException(String.format("Dispatch task to %s failed, get response failed",
taskExecutionContext.getHost()), e);
} catch (RemotingException e) {
throw new TaskDispatchException(String.format("Dispatch task to %s failed",
taskExecutionContext.getHost()), e);
}
}
protected void addDispatchEvent(TaskExecuteRunnable taskExecuteRunnable) {
TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext();
TaskEvent taskEvent = TaskEvent.newDispatchEvent(
taskExecutionContext.getProcessInstanceId(),
taskExecutionContext.getTaskInstanceId(),
taskExecutionContext.getHost());
taskEventService.addEvent(taskEvent);
}
private void addDispatchFailedEvent(TaskExecuteRunnable taskExecuteRunnable) {
TaskExecutionContext taskExecutionContext = taskExecuteRunnable.getTaskExecutionContext();
TaskEvent taskEvent = TaskEvent.builder()
.processInstanceId(taskExecutionContext.getProcessInstanceId())
.taskInstanceId(taskExecutionContext.getTaskInstanceId())
.state(TaskExecutionStatus.FAILURE)
.logPath(taskExecutionContext.getLogPath())
.executePath(taskExecutionContext.getExecutePath())
.appIds(taskExecutionContext.getAppIds())
.processId(taskExecutionContext.getProcessId())
.varPool(taskExecutionContext.getVarPool())
.startTime(DateUtils.timeStampToDate(taskExecutionContext.getStartTime()))
.endTime(new Date())
.event(TaskEventType.RESULT)
.build();
taskEventService.addEvent(taskEvent);
}
}

46
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueue.java

@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner;
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable;
import java.util.concurrent.PriorityBlockingQueue;
import lombok.extern.slf4j.Slf4j;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class GlobalTaskDispatchWaitingQueue {
private final PriorityBlockingQueue<DefaultTaskExecuteRunnable> queue = new PriorityBlockingQueue<>();
public void submitNeedToDispatchTaskExecuteRunnable(DefaultTaskExecuteRunnable priorityTaskExecuteRunnable) {
queue.put(priorityTaskExecuteRunnable);
}
public DefaultTaskExecuteRunnable takeNeedToDispatchTaskExecuteRunnable() throws InterruptedException {
return queue.take();
}
public int getWaitingDispatchTaskNumber() {
return queue.size();
}
}

99
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/GlobalTaskDispatchWaitingQueueLooper.java

@ -0,0 +1,99 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner;
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatchFactory;
import org.apache.dolphinscheduler.server.master.runner.dispatcher.TaskDispatcher;
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class GlobalTaskDispatchWaitingQueueLooper extends BaseDaemonThread implements AutoCloseable {
@Autowired
private GlobalTaskDispatchWaitingQueue globalTaskDispatchWaitingQueue;
@Autowired
private TaskDispatchFactory taskDispatchFactory;
private final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false);
private final AtomicInteger DISPATCHED_TIMES = new AtomicInteger();
private static final Integer MAX_DISPATCHED_FAILED_TIMES = 100;
public GlobalTaskDispatchWaitingQueueLooper() {
super("GlobalTaskDispatchWaitingQueueLooper");
}
@Override
public synchronized void start() {
if (!RUNNING_FLAG.compareAndSet(false, true)) {
log.error("The GlobalTaskDispatchWaitingQueueLooper already started, will not start again");
return;
}
log.info("GlobalTaskDispatchWaitingQueueLooper starting...");
super.start();
log.info("GlobalTaskDispatchWaitingQueueLooper started...");
}
@Override
public void run() {
DefaultTaskExecuteRunnable defaultTaskExecuteRunnable;
while (RUNNING_FLAG.get()) {
try {
defaultTaskExecuteRunnable = globalTaskDispatchWaitingQueue.takeNeedToDispatchTaskExecuteRunnable();
} catch (InterruptedException e) {
log.warn("Get waiting dispatch task failed, the current thread has been interrupted, will stop loop");
Thread.currentThread().interrupt();
break;
}
try {
final TaskDispatcher taskDispatcher = taskDispatchFactory
.getTaskDispatcher(defaultTaskExecuteRunnable.getTaskInstance().getTaskType());
taskDispatcher.dispatchTask(defaultTaskExecuteRunnable);
DISPATCHED_TIMES.set(0);
} catch (Exception e) {
globalTaskDispatchWaitingQueue.submitNeedToDispatchTaskExecuteRunnable(defaultTaskExecuteRunnable);
if (DISPATCHED_TIMES.incrementAndGet() > MAX_DISPATCHED_FAILED_TIMES) {
ThreadUtils.sleep(10 * 1000L);
}
log.error("Dispatch task failed", e);
}
}
log.info("GlobalTaskDispatchWaitingQueueLooper started...");
}
@Override
public void close() throws Exception {
if (RUNNING_FLAG.compareAndSet(true, false)) {
log.info("GlobalTaskDispatchWaitingQueueLooper stopping...");
log.info("GlobalTaskDispatchWaitingQueueLooper stopped...");
}
}
}

54
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterDelayTaskExecuteRunnableDelayQueue.java

@ -0,0 +1,54 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterDelayTaskExecuteRunnable;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable;
import java.util.concurrent.DelayQueue;
import org.springframework.stereotype.Component;
/**
*
*/
@Component
public class MasterDelayTaskExecuteRunnableDelayQueue {
private final DelayQueue<MasterDelayTaskExecuteRunnable> masterDelayTaskExecuteRunnableDelayQueue =
new DelayQueue<>();
public boolean submitMasterDelayTaskExecuteRunnable(MasterDelayTaskExecuteRunnable masterDelayTaskExecuteRunnable) {
return masterDelayTaskExecuteRunnableDelayQueue.offer(masterDelayTaskExecuteRunnable);
}
public MasterDelayTaskExecuteRunnable takeMasterDelayTaskExecuteRunnable() throws InterruptedException {
return masterDelayTaskExecuteRunnableDelayQueue.take();
}
// todo: if we move the delay process to master, than we don't need this method, since dispatchProcess can directly
// submit to thread pool
public boolean removeMasterDelayTaskExecuteRunnable(MasterTaskExecuteRunnable masterTaskExecuteRunnable) {
return masterDelayTaskExecuteRunnableDelayQueue.remove(masterTaskExecuteRunnable);
}
public int size() {
return masterDelayTaskExecuteRunnableDelayQueue.size();
}
}

81
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterDelayTaskExecuteRunnableDelayQueueLooper.java

@ -0,0 +1,81 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner;
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterDelayTaskExecuteRunnable;
import java.util.concurrent.atomic.AtomicBoolean;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class MasterDelayTaskExecuteRunnableDelayQueueLooper extends BaseDaemonThread implements AutoCloseable {
@Autowired
private MasterDelayTaskExecuteRunnableDelayQueue masterDelayTaskExecuteRunnableDelayQueue;
@Autowired
private MasterTaskExecuteRunnableThreadPool masterTaskExecuteRunnableThreadPool;
private final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false);
public MasterDelayTaskExecuteRunnableDelayQueueLooper() {
super("MasterDelayTaskExecuteRunnableDelayQueueLooper");
}
@Override
public synchronized void start() {
if (!RUNNING_FLAG.compareAndSet(false, true)) {
log.error("The MasterDelayTaskExecuteRunnableDelayQueueLooper already started, will not start again");
return;
}
log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper starting...");
super.start();
masterTaskExecuteRunnableThreadPool.start();
log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper started...");
}
@Override
public void run() {
while (RUNNING_FLAG.get()) {
try {
final MasterDelayTaskExecuteRunnable masterDelayTaskExecuteRunnable =
masterDelayTaskExecuteRunnableDelayQueue.takeMasterDelayTaskExecuteRunnable();
masterTaskExecuteRunnableThreadPool.submitMasterTaskExecuteRunnable(masterDelayTaskExecuteRunnable);
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();
log.warn("MasterDelayTaskExecuteRunnableDelayQueueLooper has been interrupted, will stop loop");
break;
}
}
log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper stop loop...");
}
@Override
public void close() throws Exception {
if (RUNNING_FLAG.compareAndSet(true, false)) {
log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper stopping...");
log.info("MasterDelayTaskExecuteRunnableDelayQueueLooper stopped...");
}
}
}

183
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerBootstrap.java

@ -18,40 +18,28 @@
package org.apache.dolphinscheduler.server.master.runner;
import org.apache.dolphinscheduler.common.constants.Constants;
import org.apache.dolphinscheduler.common.enums.SlotCheckState;
import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager;
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.common.utils.NetUtils;
import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.dao.repository.TaskDefinitionLogDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.event.WorkflowEvent;
import org.apache.dolphinscheduler.server.master.event.WorkflowEventQueue;
import org.apache.dolphinscheduler.server.master.event.WorkflowEventType;
import org.apache.dolphinscheduler.server.master.exception.MasterException;
import org.apache.dolphinscheduler.server.master.exception.WorkflowCreateException;
import org.apache.dolphinscheduler.server.master.metrics.MasterServerMetrics;
import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics;
import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager;
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient;
import org.apache.dolphinscheduler.service.alert.ProcessAlertManager;
import org.apache.dolphinscheduler.service.command.CommandService;
import org.apache.dolphinscheduler.service.expand.CuringParamsService;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.commons.collections4.CollectionUtils;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ThreadPoolExecutor;
import lombok.extern.slf4j.Slf4j;
@ -65,43 +53,17 @@ import org.springframework.stereotype.Service;
@Slf4j
public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCloseable {
@Autowired
private ProcessService processService;
@Autowired
private CommandService commandService;
@Autowired
private ProcessInstanceDao processInstanceDao;
@Autowired
private TaskInstanceDao taskInstanceDao;
@Autowired
private TaskDefinitionLogDao taskDefinitionLogDao;
@Autowired
private MasterConfig masterConfig;
@Autowired
private ProcessAlertManager processAlertManager;
@Autowired
private MasterRpcClient masterRpcClient;
/**
* master prepare exec service
*/
private ThreadPoolExecutor masterPrepareExecService;
@Autowired
private ProcessInstanceExecCacheManager processInstanceExecCacheManager;
@Autowired
private StateWheelExecuteThread stateWheelExecuteThread;
@Autowired
private CuringParamsService curingGlobalParamsService;
private WorkflowExecuteRunnableFactory workflowExecuteRunnableFactory;
@Autowired
private WorkflowEventQueue workflowEventQueue;
@ -112,33 +74,31 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl
@Autowired
private ServerNodeManager serverNodeManager;
private String masterAddress;
@Autowired
private MasterTaskExecutorBootstrap masterTaskExecutorBootstrap;
protected MasterSchedulerBootstrap() {
super("MasterCommandLoopThread");
}
/**
* constructor of MasterSchedulerService
*/
public void init() {
this.masterPrepareExecService = (ThreadPoolExecutor) ThreadUtils
.newDaemonFixedThreadExecutor("MasterPreExecThread", masterConfig.getPreExecThreads());
this.masterAddress = NetUtils.getAddr(masterConfig.getListenPort());
}
@Override
public synchronized void start() {
log.info("Master schedule bootstrap starting..");
log.info("MasterSchedulerBootstrap starting..");
super.start();
workflowEventLooper.start();
log.info("Master schedule bootstrap started...");
masterTaskExecutorBootstrap.start();
log.info("MasterSchedulerBootstrap started...");
}
@Override
public void close() {
log.info("Master schedule bootstrap stopping...");
log.info("Master schedule bootstrap stopped...");
public void close() throws Exception {
log.info("MasterSchedulerBootstrap stopping...");
try (
final WorkflowEventLooper workflowEventLooper1 = workflowEventLooper;
final MasterTaskExecutorBootstrap masterTaskExecutorBootstrap1 = masterTaskExecutorBootstrap) {
// closed the resource
}
log.info("MasterSchedulerBootstrap stopped...");
}
/**
@ -150,15 +110,14 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl
try {
if (!ServerLifeCycleManager.isRunning()) {
// the current server is not at running status, cannot consume command.
log.warn("The current server {} is not at running status, cannot consumes commands.",
this.masterAddress);
log.warn("The current server is not at running status, cannot consumes commands.");
Thread.sleep(Constants.SLEEP_TIME_MILLIS);
}
// todo: if the workflow event queue is much, we need to handle the back pressure
boolean isOverload =
OSUtils.isOverload(masterConfig.getMaxCpuLoadAvg(), masterConfig.getReservedMemory());
if (isOverload) {
log.warn("The current server {} is overload, cannot consumes commands.", this.masterAddress);
log.warn("The current server is overload, cannot consumes commands.");
MasterServerMetrics.incMasterOverload();
Thread.sleep(Constants.SLEEP_TIME_MILLIS);
continue;
@ -170,39 +129,25 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl
continue;
}
List<ProcessInstance> processInstances = command2ProcessInstance(commands);
if (CollectionUtils.isEmpty(processInstances)) {
// indicate that the command transform to processInstance error, sleep for 1s
Thread.sleep(Constants.SLEEP_TIME_MILLIS);
continue;
}
commands.parallelStream()
.forEach(command -> {
try {
WorkflowExecuteRunnable workflowExecuteRunnable =
workflowExecuteRunnableFactory.createWorkflowExecuteRunnable(command);
ProcessInstance processInstance = workflowExecuteRunnable.getProcessInstance();
if (processInstanceExecCacheManager.contains(processInstance.getId())) {
log.error(
"The workflow instance is already been cached, this case shouldn't be happened");
}
processInstanceExecCacheManager.cache(processInstance.getId(), workflowExecuteRunnable);
workflowEventQueue.addEvent(
new WorkflowEvent(WorkflowEventType.START_WORKFLOW, processInstance.getId()));
} catch (WorkflowCreateException workflowCreateException) {
log.error("Master handle command {} error ", command.getId(), workflowCreateException);
commandService.moveToErrorCommand(command, workflowCreateException.toString());
}
});
MasterServerMetrics.incMasterConsumeCommand(commands.size());
processInstances.forEach(processInstance -> {
try {
LogUtils.setWorkflowInstanceIdMDC(processInstance.getId());
if (processInstanceExecCacheManager.contains(processInstance.getId())) {
log.error(
"The workflow instance is already been cached, this case shouldn't be happened");
}
WorkflowExecuteRunnable workflowRunnable = new WorkflowExecuteRunnable(processInstance,
commandService,
processService,
processInstanceDao,
masterRpcClient,
processAlertManager,
masterConfig,
stateWheelExecuteThread,
curingGlobalParamsService,
taskInstanceDao,
taskDefinitionLogDao);
processInstanceExecCacheManager.cache(processInstance.getId(), workflowRunnable);
workflowEventQueue.addEvent(new WorkflowEvent(WorkflowEventType.START_WORKFLOW,
processInstance.getId()));
} finally {
LogUtils.removeWorkflowInstanceIdMDC();
}
});
} catch (InterruptedException interruptedException) {
log.warn("Master schedule bootstrap interrupted, close the loop", interruptedException);
Thread.currentThread().interrupt();
@ -215,50 +160,6 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl
}
}
private List<ProcessInstance> command2ProcessInstance(List<Command> commands) throws InterruptedException {
long commandTransformStartTime = System.currentTimeMillis();
log.info("Master schedule bootstrap transforming command to ProcessInstance, commandSize: {}",
commands.size());
List<ProcessInstance> processInstances = Collections.synchronizedList(new ArrayList<>(commands.size()));
CountDownLatch latch = new CountDownLatch(commands.size());
for (final Command command : commands) {
masterPrepareExecService.execute(() -> {
try {
// Note: this check is not safe, the slot may change after command transform.
// We use the database transaction in `handleCommand` so that we can guarantee the command will
// always be executed
// by only one master
SlotCheckState slotCheckState = slotCheck(command);
if (slotCheckState.equals(SlotCheckState.CHANGE) || slotCheckState.equals(SlotCheckState.INJECT)) {
log.info("Master handle command {} skip, slot check state: {}", command.getId(),
slotCheckState);
return;
}
ProcessInstance processInstance = processService.handleCommand(masterAddress, command);
if (processInstance != null) {
processInstances.add(processInstance);
log.info("Master handle command {} end, create process instance {}", command.getId(),
processInstance.getId());
}
} catch (Exception e) {
log.error("Master handle command {} error ", command.getId(), e);
commandService.moveToErrorCommand(command, e.toString());
} finally {
latch.countDown();
}
});
}
// make sure to finish handling command each time before next scan
latch.await();
log.info(
"Master schedule bootstrap transformed command to ProcessInstance, commandSize: {}, processInstanceSize: {}",
commands.size(), processInstances.size());
ProcessInstanceMetrics
.recordProcessInstanceGenerateTime(System.currentTimeMillis() - commandTransformStartTime);
return processInstances;
}
private List<Command> findCommands() throws MasterException {
try {
long scheduleStartTime = System.currentTimeMillis();
@ -284,18 +185,4 @@ public class MasterSchedulerBootstrap extends BaseDaemonThread implements AutoCl
}
}
private SlotCheckState slotCheck(Command command) {
int slot = serverNodeManager.getSlot();
int masterSize = serverNodeManager.getMasterSize();
SlotCheckState state;
if (masterSize <= 0) {
state = SlotCheckState.CHANGE;
} else if (command.getId() % masterSize == slot) {
state = SlotCheckState.PASS;
} else {
state = SlotCheckState.INJECT;
}
return state;
}
}

89
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecuteRunnableThreadPool.java

@ -0,0 +1,89 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.runner.execute.MasterTaskExecuteRunnable;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
@Slf4j
@Component
public class MasterTaskExecuteRunnableThreadPool {
@Autowired
private MasterConfig masterConfig;
private static final Map<Integer, MasterTaskExecuteRunnable> SUBMITTED_MASTER_TASK_MAP = new ConcurrentHashMap<>();
private ListeningExecutorService listeningExecutorService;
public synchronized void start() {
log.info("MasterTaskExecuteRunnableThreadPool starting...");
this.listeningExecutorService = MoreExecutors.listeningDecorator(ThreadUtils.newDaemonFixedThreadExecutor(
"MasterTaskExecuteRunnableThread", masterConfig.getMasterTaskExecuteThreadPoolSize()));
log.info("MasterTaskExecuteRunnableThreadPool started...");
}
public void submitMasterTaskExecuteRunnable(MasterTaskExecuteRunnable masterTaskExecuteRunnable) {
ListenableFuture<?> future = listeningExecutorService.submit(masterTaskExecuteRunnable);
Futures.addCallback(future, new MasterTaskExecuteCallback(masterTaskExecuteRunnable),
this.listeningExecutorService);
SUBMITTED_MASTER_TASK_MAP.put(masterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId(),
masterTaskExecuteRunnable);
}
public MasterTaskExecuteRunnable getMasterTaskExecuteRunnable(Integer taskInstanceId) {
return SUBMITTED_MASTER_TASK_MAP.get(taskInstanceId);
}
private static class MasterTaskExecuteCallback implements FutureCallback {
private MasterTaskExecuteRunnable masterTaskExecuteRunnable;
public MasterTaskExecuteCallback(MasterTaskExecuteRunnable masterTaskExecuteRunnable) {
this.masterTaskExecuteRunnable = masterTaskExecuteRunnable;
}
@Override
public void onSuccess(Object result) {
log.info("MasterTaskExecuteRunnable execute success, will remove this task");
SUBMITTED_MASTER_TASK_MAP.remove(masterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId());
}
@Override
public void onFailure(Throwable t) {
log.info("MasterTaskExecuteRunnable execute failed, will remove this task");
SUBMITTED_MASTER_TASK_MAP.remove(masterTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId());
}
}
}

62
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterTaskExecutorBootstrap.java

@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner;
import org.apache.dolphinscheduler.server.master.runner.execute.AsyncMasterTaskDelayQueueLooper;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class MasterTaskExecutorBootstrap implements AutoCloseable {
@Autowired
private GlobalTaskDispatchWaitingQueueLooper globalTaskDispatchWaitingQueueLooper;
@Autowired
private MasterDelayTaskExecuteRunnableDelayQueueLooper masterDelayTaskExecuteRunnableDelayQueueLooper;
@Autowired
private AsyncMasterTaskDelayQueueLooper asyncMasterTaskDelayQueueLooper;
public synchronized void start() {
log.info("MasterTaskExecutorBootstrap starting...");
globalTaskDispatchWaitingQueueLooper.start();
masterDelayTaskExecuteRunnableDelayQueueLooper.start();
asyncMasterTaskDelayQueueLooper.start();
log.info("MasterTaskExecutorBootstrap started...");
}
@Override
public void close() throws Exception {
log.info("MasterTaskExecutorBootstrap closing...");
try (
final GlobalTaskDispatchWaitingQueueLooper globalTaskDispatchWaitingQueueLooper1 =
globalTaskDispatchWaitingQueueLooper;
final MasterDelayTaskExecuteRunnableDelayQueueLooper masterDelayTaskExecuteRunnableDelayQueueLooper1 =
masterDelayTaskExecuteRunnableDelayQueueLooper;
final AsyncMasterTaskDelayQueueLooper asyncMasterTaskDelayQueueLooper1 =
asyncMasterTaskDelayQueueLooper) {
// closed the resource
}
log.info("MasterTaskExecutorBootstrap closed...");
}
}

14
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StateWheelExecuteThread.java

@ -136,8 +136,9 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
return;
}
for (Integer processInstanceId : processInstanceTimeoutCheckList) {
try {
LogUtils.setWorkflowInstanceIdMDC(processInstanceId);
try (
LogUtils.MDCAutoClosableContext mdcAutoClosableContext =
LogUtils.setWorkflowInstanceIdMDC(processInstanceId)) {
WorkflowExecuteRunnable workflowExecuteThread = processInstanceExecCacheManager.getByProcessInstanceId(
processInstanceId);
if (workflowExecuteThread == null) {
@ -162,8 +163,6 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
}
} catch (Exception ex) {
log.error("Check workflow instance timeout error");
} finally {
LogUtils.removeWorkflowInstanceIdMDC();
}
}
}
@ -246,9 +245,10 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
return;
}
for (TaskInstanceKey taskInstanceKey : taskInstanceTimeoutCheckList) {
try {
try (
LogUtils.MDCAutoClosableContext mdcAutoClosableContext =
LogUtils.setWorkflowInstanceIdMDC(taskInstanceKey.getProcessInstanceId())) {
int processInstanceId = taskInstanceKey.getProcessInstanceId();
LogUtils.setWorkflowInstanceIdMDC(processInstanceId);
long taskCode = taskInstanceKey.getTaskCode();
WorkflowExecuteRunnable workflowExecuteThread =
@ -282,8 +282,6 @@ public class StateWheelExecuteThread extends BaseDaemonThread {
}
} catch (Exception ex) {
log.error("Check task timeout error, taskInstanceKey: {}", taskInstanceKey, ex);
} finally {
LogUtils.removeWorkflowInstanceIdMDC();
}
}
}

63
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/StreamTaskExecuteRunnable.java

@ -42,20 +42,18 @@ import org.apache.dolphinscheduler.plugin.task.api.parameters.ParametersNode;
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
import org.apache.dolphinscheduler.plugin.task.api.parser.ParamUtils;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.remote.command.task.TaskDispatchMessage;
import org.apache.dolphinscheduler.remote.command.task.TaskExecuteRunningMessageAck;
import org.apache.dolphinscheduler.remote.command.task.TaskExecuteStartMessage;
import org.apache.dolphinscheduler.server.master.builder.TaskExecutionContextBuilder;
import org.apache.dolphinscheduler.server.master.cache.StreamTaskInstanceExecCacheManager;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher;
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
import org.apache.dolphinscheduler.server.master.dispatch.enums.ExecutorType;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.ExecuteException;
import org.apache.dolphinscheduler.server.master.event.StateEventHandleError;
import org.apache.dolphinscheduler.server.master.event.StateEventHandleException;
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEvent;
import org.apache.dolphinscheduler.server.master.runner.dispatcher.WorkerTaskDispatcher;
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable;
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnableFactory;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.spi.enums.ResourceType;
@ -88,7 +86,9 @@ public class StreamTaskExecuteRunnable implements Runnable {
protected TaskInstanceDao taskInstanceDao;
protected ExecutorDispatcher dispatcher;
protected DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory;
protected WorkerTaskDispatcher workerTaskDispatcher;
protected ProcessTaskRelationMapper processTaskRelationMapper;
@ -114,7 +114,7 @@ public class StreamTaskExecuteRunnable implements Runnable {
public StreamTaskExecuteRunnable(TaskDefinition taskDefinition, TaskExecuteStartMessage taskExecuteStartMessage) {
this.processService = SpringApplicationContext.getBean(ProcessService.class);
this.masterConfig = SpringApplicationContext.getBean(MasterConfig.class);
this.dispatcher = SpringApplicationContext.getBean(ExecutorDispatcher.class);
this.workerTaskDispatcher = SpringApplicationContext.getBean(WorkerTaskDispatcher.class);
this.taskPluginManager = SpringApplicationContext.getBean(TaskPluginManager.class);
this.processTaskRelationMapper = SpringApplicationContext.getBean(ProcessTaskRelationMapper.class);
this.taskInstanceDao = SpringApplicationContext.getBean(TaskInstanceDao.class);
@ -144,48 +144,20 @@ public class StreamTaskExecuteRunnable implements Runnable {
int processDefinitionVersion = processTaskRelationList.get(0).getProcessDefinitionVersion();
processDefinition = processService.findProcessDefinition(processDefinitionCode, processDefinitionVersion);
// dispatch task
TaskExecutionContext taskExecutionContext = getTaskExecutionContext(taskInstance);
if (taskExecutionContext == null) {
taskInstance.setState(TaskExecutionStatus.FAILURE);
taskInstanceDao.upsertTaskInstance(taskInstance);
return;
}
TaskDispatchMessage dispatchCommand = new TaskDispatchMessage(taskExecutionContext,
masterConfig.getMasterAddress(),
taskExecutionContext.getHost(),
System.currentTimeMillis());
ExecutionContext executionContext = new ExecutionContext(dispatchCommand.convert2Command(), ExecutorType.WORKER,
taskExecutionContext.getWorkerGroup(), taskInstance);
Boolean dispatchSuccess = false;
try {
dispatcher.dispatch(executionContext);
dispatchSuccess = true;
} catch (ExecuteException e) {
log.error("Master dispatch task to worker error, taskInstanceId: {}, worker: {}",
taskInstance.getId(),
executionContext.getHost(),
e);
}
if (!dispatchSuccess) {
log.info("Master failed to dispatch task to worker, taskInstanceId: {}, worker: {}",
taskInstance.getId(),
executionContext.getHost());
// set task instance fail
DefaultTaskExecuteRunnable taskExecuteRunnable =
defaultTaskExecuteRunnableFactory.createTaskExecuteRunnable(taskInstance);
workerTaskDispatcher.dispatchTask(taskExecuteRunnable);
} catch (Exception e) {
log.error("Master dispatch task to worker error, taskInstanceName: {}", taskInstance.getName(), e);
taskInstance.setState(TaskExecutionStatus.FAILURE);
taskInstanceDao.upsertTaskInstance(taskInstance);
return;
}
// set started flag
taskRunnableStatus = TaskRunnableStatus.STARTED;
log.info("Master success dispatch task to worker, taskInstanceId: {}, worker: {}",
taskInstance.getId(),
executionContext.getHost());
log.info("Master success dispatch task to worker, taskInstanceName: {}, worker: {}", taskInstance.getId(),
taskInstance.getHost());
}
public boolean isStart() {
@ -344,8 +316,11 @@ public class StreamTaskExecuteRunnable implements Runnable {
TaskChannel taskChannel = taskPluginManager.getTaskChannel(taskInstance.getTaskType());
ResourceParametersHelper resources = taskChannel.getResources(taskInstance.getTaskParams());
AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder()
.taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
AbstractParameters baseParam = taskPluginManager.getParameters(
ParametersNode.builder()
.taskType(taskInstance.getTaskType())
.taskParams(taskInstance.getTaskParams())
.build());
Map<String, Property> propertyMap = paramParsingPreparation(taskInstance, baseParam);
TaskExecutionContext taskExecutionContext = TaskExecutionContextBuilder.get()
.buildWorkflowInstanceHost(masterConfig.getMasterAddress())

57
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowEventLooper.java

@ -18,7 +18,6 @@
package org.apache.dolphinscheduler.server.master.runner;
import org.apache.dolphinscheduler.common.constants.Constants;
import org.apache.dolphinscheduler.common.lifecycle.ServerLifeCycleManager;
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
@ -32,6 +31,7 @@ import org.apache.dolphinscheduler.server.master.event.WorkflowEventType;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import javax.annotation.PostConstruct;
@ -42,7 +42,7 @@ import org.springframework.stereotype.Component;
@Component
@Slf4j
public class WorkflowEventLooper extends BaseDaemonThread {
public class WorkflowEventLooper extends BaseDaemonThread implements AutoCloseable {
@Autowired
private WorkflowEventQueue workflowEventQueue;
@ -52,6 +52,8 @@ public class WorkflowEventLooper extends BaseDaemonThread {
private final Map<WorkflowEventType, WorkflowEventHandler> workflowEventHandlerMap = new HashMap<>();
private final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false);
protected WorkflowEventLooper() {
super("WorkflowEventLooper");
}
@ -65,44 +67,59 @@ public class WorkflowEventLooper extends BaseDaemonThread {
@Override
public synchronized void start() {
log.info("WorkflowEventLooper thread starting");
if (!RUNNING_FLAG.compareAndSet(false, true)) {
log.error("WorkflowEventLooper thread has already started, will not start again");
return;
}
log.info("WorkflowEventLooper starting...");
super.start();
log.info("WorkflowEventLooper thread started");
log.info("WorkflowEventLooper started...");
}
public void run() {
WorkflowEvent workflowEvent = null;
while (!ServerLifeCycleManager.isStopped()) {
WorkflowEvent workflowEvent;
while (RUNNING_FLAG.get()) {
try {
workflowEvent = workflowEventQueue.poolEvent();
LogUtils.setWorkflowInstanceIdMDC(workflowEvent.getWorkflowInstanceId());
log.info("Workflow event looper receive a workflow event: {}, will handle this", workflowEvent);
WorkflowEventHandler workflowEventHandler =
workflowEventHandlerMap.get(workflowEvent.getWorkflowEventType());
workflowEventHandler.handleWorkflowEvent(workflowEvent);
} catch (InterruptedException e) {
log.warn("WorkflowEventLooper thread is interrupted, will close this loop", e);
log.warn("WorkflowEventLooper thread is interrupted, will close this loop");
Thread.currentThread().interrupt();
break;
}
try (
LogUtils.MDCAutoClosableContext mdcAutoClosableContext =
LogUtils.setWorkflowInstanceIdMDC(workflowEvent.getWorkflowInstanceId())) {
log.info("Begin to handle WorkflowEvent: {}", workflowEvent);
WorkflowEventHandler workflowEventHandler =
workflowEventHandlerMap.get(workflowEvent.getWorkflowEventType());
workflowEventHandler.handleWorkflowEvent(workflowEvent);
log.info("Success handle WorkflowEvent: {}", workflowEvent);
} catch (WorkflowEventHandleException workflowEventHandleException) {
log.error("Handle workflow event failed, will add this event to event queue again, event: {}",
workflowEvent, workflowEventHandleException);
log.error("Handle workflow event failed, will retry again: {}", workflowEvent,
workflowEventHandleException);
workflowEventQueue.addEvent(workflowEvent);
ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS);
} catch (WorkflowEventHandleError workflowEventHandleError) {
log.error("Handle workflow event error, will drop this event, event: {}",
log.error("Handle workflow event error, will drop this event: {}",
workflowEvent,
workflowEventHandleError);
} catch (Exception unknownException) {
log.error(
"Handle workflow event failed, get a unknown exception, will add this event to event queue again, event: {}",
workflowEvent, unknownException);
log.error("Handle workflow event failed, get a unknown exception, will retry again: {}", workflowEvent,
unknownException);
workflowEventQueue.addEvent(workflowEvent);
ThreadUtils.sleep(Constants.SLEEP_TIME_MILLIS);
} finally {
LogUtils.removeWorkflowInstanceIdMDC();
}
}
}
@Override
public void close() throws Exception {
if (!RUNNING_FLAG.compareAndSet(true, false)) {
log.info("WorkflowEventLooper thread is not start, no need to close");
return;
}
log.info("WorkflowEventLooper is closing...");
this.interrupt();
log.info("WorkflowEventLooper closed...");
}
}

340
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnable.java

@ -66,6 +66,7 @@ import org.apache.dolphinscheduler.plugin.task.api.enums.DependResult;
import org.apache.dolphinscheduler.plugin.task.api.enums.Direct;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.remote.command.Message;
import org.apache.dolphinscheduler.remote.command.task.TaskWakeupRequest;
@ -82,11 +83,12 @@ import org.apache.dolphinscheduler.server.master.event.StateEventHandler;
import org.apache.dolphinscheduler.server.master.event.StateEventHandlerManager;
import org.apache.dolphinscheduler.server.master.event.TaskStateEvent;
import org.apache.dolphinscheduler.server.master.event.WorkflowStateEvent;
import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException;
import org.apache.dolphinscheduler.server.master.metrics.TaskMetrics;
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient;
import org.apache.dolphinscheduler.server.master.runner.task.ITaskProcessor;
import org.apache.dolphinscheduler.server.master.runner.task.TaskAction;
import org.apache.dolphinscheduler.server.master.runner.task.TaskProcessorFactory;
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable;
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnableFactory;
import org.apache.dolphinscheduler.server.master.utils.TaskUtils;
import org.apache.dolphinscheduler.service.alert.ProcessAlertManager;
import org.apache.dolphinscheduler.service.command.CommandService;
import org.apache.dolphinscheduler.service.cron.CronUtils;
@ -103,7 +105,6 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.math.NumberUtils;
import org.apache.commons.lang3.tuple.Pair;
import java.lang.reflect.InvocationTargetException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@ -118,6 +119,7 @@ import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicBoolean;
@ -181,10 +183,9 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
private final Map<Integer, TaskInstance> taskInstanceMap = new ConcurrentHashMap<>();
/**
* running taskProcessor, taskCode as key, taskProcessor as value
* only on taskProcessor per taskCode
* TaskCode as Key, TaskExecuteRunnable as Value
*/
private final Map<Long, ITaskProcessor> activeTaskProcessorMaps = new ConcurrentHashMap<>();
private final Map<Long, DefaultTaskExecuteRunnable> taskExecuteRunnableMap = new ConcurrentHashMap<>();
/**
* valid task map, taskCode as key, taskId as value
@ -246,6 +247,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
private final String masterAddress;
private final DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory;
/**
* @param processInstance processInstance
* @param processService processService
@ -266,7 +269,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
@NonNull StateWheelExecuteThread stateWheelExecuteThread,
@NonNull CuringParamsService curingParamsService,
@NonNull TaskInstanceDao taskInstanceDao,
@NonNull TaskDefinitionLogDao taskDefinitionLogDao) {
@NonNull TaskDefinitionLogDao taskDefinitionLogDao,
@NonNull DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory) {
this.processService = processService;
this.commandService = commandService;
this.processInstanceDao = processInstanceDao;
@ -278,6 +282,10 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
this.taskInstanceDao = taskInstanceDao;
this.taskDefinitionLogDao = taskDefinitionLogDao;
this.masterAddress = NetUtils.getAddr(masterConfig.getListenPort());
this.defaultTaskExecuteRunnableFactory = defaultTaskExecuteRunnableFactory;
this.processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion());
this.processInstance.setProcessDefinition(processDefinition);
TaskMetrics.registerTaskPrepared(readyToSubmitTaskQueue::size);
}
@ -377,24 +385,30 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
}
public boolean checkForceStartAndWakeUp(StateEvent stateEvent) {
TaskGroupQueue taskGroupQueue = this.processService.loadTaskGroupQueue(stateEvent.getTaskInstanceId());
TaskGroupQueue taskGroupQueue = processService.loadTaskGroupQueue(stateEvent.getTaskInstanceId());
if (taskGroupQueue.getForceStart() == Flag.YES.getCode()) {
log.info("Begin to force start taskGroupQueue: {}", taskGroupQueue.getId());
TaskInstance taskInstance = this.taskInstanceDao.findTaskInstanceById(stateEvent.getTaskInstanceId());
ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskInstance.getTaskCode());
taskProcessor.action(TaskAction.DISPATCH);
this.processService.updateTaskGroupQueueStatus(taskGroupQueue.getTaskId(),
TaskGroupQueueStatus.ACQUIRE_SUCCESS.getCode());
log.info("Success force start taskGroupQueue: {}", taskGroupQueue.getId());
TaskInstance taskInstance = taskInstanceDao.findTaskInstanceById(stateEvent.getTaskInstanceId());
DefaultTaskExecuteRunnable defaultTaskExecuteRunnable =
taskExecuteRunnableMap.get(taskInstance.getTaskCode());
if (defaultTaskExecuteRunnable != null) {
defaultTaskExecuteRunnable.dispatch();
this.processService.updateTaskGroupQueueStatus(taskGroupQueue.getTaskId(),
TaskGroupQueueStatus.ACQUIRE_SUCCESS.getCode());
log.info("Success force start task: {}, taskGroup: {}", taskGroupQueue.getTaskName(),
taskGroupQueue.getGroupId());
} else {
log.warn("Cannot find the TaskExecuteRunnable: {}", taskGroupQueue.getTaskName());
}
return true;
}
if (taskGroupQueue.getInQueue() == Flag.YES.getCode()) {
log.info("Begin to wake up taskGroupQueue: {}", taskGroupQueue.getId());
boolean acquireTaskGroup = processService.robTaskGroupResource(taskGroupQueue);
if (acquireTaskGroup) {
TaskInstance taskInstance = this.taskInstanceDao.findTaskInstanceById(stateEvent.getTaskInstanceId());
ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskInstance.getTaskCode());
taskProcessor.action(TaskAction.DISPATCH);
TaskInstance taskInstance = taskInstanceDao.findTaskInstanceById(stateEvent.getTaskInstanceId());
taskExecuteRunnableMap.get(taskInstance.getTaskCode()).dispatch();
log.info("Success wake up taskGroupQueue: {}", taskGroupQueue.getId());
return true;
}
@ -422,7 +436,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
log.info("TaskInstance finished task code:{} state:{}", taskInstance.getTaskCode(), taskInstance.getState());
try {
activeTaskProcessorMaps.remove(taskInstance.getTaskCode());
// Do we need to remove?
taskExecuteRunnableMap.remove(taskInstance.getTaskCode());
stateWheelExecuteThread.removeTask4TimeoutCheck(processInstance, taskInstance);
stateWheelExecuteThread.removeTask4RetryCheck(processInstance, taskInstance);
stateWheelExecuteThread.removeTask4StateCheck(processInstance, taskInstance);
@ -721,14 +736,15 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
*/
@Override
public WorkflowSubmitStatus call() {
if (isStart()) {
// This case should not been happened
log.warn("[WorkflowInstance-{}] The workflow has already been started", processInstance.getId());
return WorkflowSubmitStatus.DUPLICATED_SUBMITTED;
}
try {
LogUtils.setWorkflowInstanceIdMDC(processInstance.getId());
try (
LogUtils.MDCAutoClosableContext mdcAutoClosableContext =
LogUtils.setWorkflowInstanceIdMDC(processInstance.getId())) {
if (isStart()) {
// This case should not been happened
log.warn("The workflow has already been started, current state: {}", workflowRunnableStatus);
return WorkflowSubmitStatus.DUPLICATED_SUBMITTED;
}
if (workflowRunnableStatus == WorkflowRunnableStatus.CREATED) {
buildFlowDag();
workflowRunnableStatus = WorkflowRunnableStatus.INITIALIZE_DAG;
@ -748,8 +764,6 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
} catch (Exception e) {
log.error("Start workflow error", e);
return WorkflowSubmitStatus.FAILED;
} finally {
LogUtils.removeWorkflowInstanceIdMDC();
}
}
@ -805,15 +819,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
commandService.createCommand(command);
}
/**
* Generate process dag
*
* @throws Exception exception
*/
// todo: move the initialize code to constructor
private void buildFlowDag() throws Exception {
processDefinition = processService.findProcessDefinition(processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion());
processInstance.setProcessDefinition(processDefinition);
List<TaskInstance> recoverNodeList = getRecoverTaskInstanceList(processInstance.getCommandParam());
@ -852,7 +859,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
private void initTaskQueue() throws StateEventHandleException, CronParseException {
taskFailedSubmit = false;
activeTaskProcessorMaps.clear();
// do we need to clear?
taskExecuteRunnableMap.clear();
dependFailedTaskSet.clear();
completeTaskMap.clear();
errorTaskMap.clear();
@ -967,100 +975,68 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
errorTaskMap);
}
/**
* submit task to execute
*
* @param taskInstance task instance
* @return TaskInstance
*/
private Optional<TaskInstance> submitTaskExec(TaskInstance taskInstance) {
private boolean executeTask(TaskInstance taskInstance) {
try {
// package task instance before submit
// package task instance before submit, inject the process instance to task instance
// todo: we need to use task execute context rather than packege a lot of pojo into task instance
// 1. submit the task instance to db
processService.packageTaskInstance(taskInstance, processInstance);
ITaskProcessor taskProcessor = TaskProcessorFactory.getTaskProcessor(taskInstance.getTaskType());
taskProcessor.init(taskInstance, processInstance);
boolean submit = taskProcessor.action(TaskAction.SUBMIT);
if (!submit) {
log.error("Submit standby task failed!, taskCode: {}, taskName: {}",
taskInstance.getTaskCode(),
taskInstance.getName());
return Optional.empty();
// todo: remove this method
if (!processService.submitTask(processInstance, taskInstance)) {
log.error("Submit standby task: {} failed", taskInstance.getName());
return true;
}
// 2. create task execute runnable
// in a dag, only one taskInstance is valid per taskCode, so need to set the old taskInstance invalid
LogUtils.setWorkflowAndTaskInstanceIDMDC(taskInstance.getProcessInstanceId(), taskInstance.getId());
if (validTaskMap.containsKey(taskInstance.getTaskCode())) {
int oldTaskInstanceId = validTaskMap.get(taskInstance.getTaskCode());
if (taskInstance.getId() != oldTaskInstanceId) {
TaskInstance oldTaskInstance = taskInstanceMap.get(oldTaskInstanceId);
oldTaskInstance.setFlag(Flag.NO);
taskInstanceDao.updateTaskInstance(oldTaskInstance);
validTaskMap.remove(taskInstance.getTaskCode());
activeTaskProcessorMaps.remove(taskInstance.getTaskCode());
try (
LogUtils.MDCAutoClosableContext mdcAutoClosableContext =
LogUtils.setTaskInstanceIdMDC(taskInstance.getId())) {
DefaultTaskExecuteRunnable taskExecuteRunnable =
defaultTaskExecuteRunnableFactory.createTaskExecuteRunnable(taskInstance);
if (validTaskMap.containsKey(taskInstance.getTaskCode())) {
int oldTaskInstanceId = validTaskMap.get(taskInstance.getTaskCode());
if (taskInstance.getId() != oldTaskInstanceId) {
TaskInstance oldTaskInstance = taskInstanceMap.get(oldTaskInstanceId);
oldTaskInstance.setFlag(Flag.NO);
taskInstanceDao.updateTaskInstance(oldTaskInstance);
validTaskMap.remove(taskInstance.getTaskCode());
taskExecuteRunnableMap.remove(taskInstance.getTaskCode());
}
}
}
validTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId());
taskInstanceMap.put(taskInstance.getId(), taskInstance);
activeTaskProcessorMaps.put(taskInstance.getTaskCode(), taskProcessor);
// if we use task group, then need to acquire the task group resource
// if there is no resource the current task instance will not be dispatched
// it will be wakeup when other tasks release the resource.
int taskGroupId = taskInstance.getTaskGroupId();
if (taskGroupId > 0) {
boolean acquireTaskGroup = processService.acquireTaskGroup(taskInstance.getId(),
taskInstance.getName(),
taskGroupId,
taskInstance.getProcessInstanceId(),
taskInstance.getTaskGroupPriority());
if (!acquireTaskGroup) {
log.info(
"Submitted task will not be dispatch right now because the first time to try to acquire" +
" task group failed, taskInstanceName: {}, taskGroupId: {}",
taskInstance.getName(), taskGroupId);
return Optional.of(taskInstance);
validTaskMap.put(taskInstance.getTaskCode(), taskInstance.getId());
taskInstanceMap.put(taskInstance.getId(), taskInstance);
taskExecuteRunnableMap.put(taskInstance.getTaskCode(), taskExecuteRunnable);
// 3. acquire the task group.
// if we use task group, then need to acquire the task group resource
// if there is no resource the current task instance will not be dispatched
// it will be wakeup when other tasks release the resource.
int taskGroupId = taskInstance.getTaskGroupId();
if (taskGroupId > 0) {
boolean acquireTaskGroup = processService.acquireTaskGroup(taskInstance.getId(),
taskInstance.getName(),
taskGroupId,
taskInstance.getProcessInstanceId(),
taskInstance.getTaskGroupPriority());
if (!acquireTaskGroup) {
log.info(
"Submitted task will not be dispatch right now because the first time to try to acquire"
+
" task group failed, taskInstanceName: {}, taskGroupId: {}",
taskInstance.getName(), taskGroupId);
return true;
}
}
}
boolean dispatchSuccess = taskProcessor.action(TaskAction.DISPATCH);
if (!dispatchSuccess) {
log.error("Dispatch standby process {} task {} failed", processInstance.getName(),
taskInstance.getName());
return Optional.empty();
}
taskProcessor.action(TaskAction.RUN);
// 4. submit to dispatch queue
taskExecuteRunnable.dispatch();
stateWheelExecuteThread.addTask4TimeoutCheck(processInstance, taskInstance);
stateWheelExecuteThread.addTask4StateCheck(processInstance, taskInstance);
if (taskProcessor.taskInstance().getState().isFinished()) {
if (processInstance.isBlocked()) {
TaskStateEvent processBlockEvent = TaskStateEvent.builder()
.processInstanceId(processInstance.getId())
.taskInstanceId(taskInstance.getId())
.status(taskProcessor.taskInstance().getState())
.type(StateEventType.PROCESS_BLOCKED)
.build();
this.stateEvents.add(processBlockEvent);
}
TaskStateEvent taskStateChangeEvent = TaskStateEvent.builder()
.processInstanceId(processInstance.getId())
.taskInstanceId(taskInstance.getId())
.status(taskProcessor.taskInstance().getState())
.type(StateEventType.TASK_STATE_CHANGE)
.build();
this.stateEvents.add(taskStateChangeEvent);
stateWheelExecuteThread.addTask4TimeoutCheck(processInstance, taskInstance);
stateWheelExecuteThread.addTask4StateCheck(processInstance, taskInstance);
return true;
}
return Optional.of(taskInstance);
} catch (Exception e) {
log.error("Submit standby task {} error, taskCode: {}", taskInstance.getName(),
taskInstance.getTaskCode(), e);
return Optional.empty();
} finally {
LogUtils.removeWorkflowAndTaskInstanceIdMDC();
log.error("Submit standby task {} error", taskInstance.getName(), e);
return false;
}
}
@ -1372,10 +1348,9 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
}
}
taskInstances.add(existTaskInstance);
continue;
} else {
taskInstances.add(createTaskInstance(processInstance, taskNodeObject));
}
TaskInstance task = createTaskInstance(processInstance, taskNodeObject);
taskInstances.add(task);
}
// the end node of the branch of the dag
if (StringUtils.isNotEmpty(parentNodeCode) && dag.getEndNode().contains(parentNodeCode)) {
@ -1398,7 +1373,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
for (TaskInstance task : taskInstances) {
if (readyToSubmitTaskQueue.contains(task)) {
log.warn("Task is already at submit queue, taskInstanceId: {}", task.getId());
log.warn("Task is already at submit queue, taskInstanceName: {}", task.getName());
continue;
}
@ -1418,7 +1393,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
}
private boolean tryToTakeOverTaskInstance(TaskInstance taskInstance) {
if (TaskProcessorFactory.isMasterTask(taskInstance.getTaskType())) {
if (TaskUtils.isMasterTask(taskInstance.getTaskType())) {
return false;
}
try {
@ -1440,16 +1415,15 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
return false;
}
ITaskProcessor taskProcessor = TaskProcessorFactory.getTaskProcessor(taskInstance.getTaskType());
taskProcessor.init(taskInstance, processInstance);
// todo: create the takeover task execute runnable.
taskExecuteRunnableMap.put(taskInstance.getTaskCode(),
defaultTaskExecuteRunnableFactory.createTaskExecuteRunnable(taskInstance));
taskInstanceMap.put(taskInstance.getId(), taskInstance);
stateWheelExecuteThread.addTask4TimeoutCheck(processInstance, taskInstance);
stateWheelExecuteThread.addTask4RetryCheck(processInstance, taskInstance);
activeTaskProcessorMaps.put(taskInstance.getTaskCode(), taskProcessor);
return true;
} catch (RemotingException | InterruptedException | InstantiationException | IllegalAccessException
| InvocationTargetException e) {
} catch (RemotingException | InterruptedException | TaskExecuteRunnableCreateException e) {
log.error(
"Takeover task instance failed, the worker {} might not be alive, will try to create a new task instance",
taskInstance.getHost(), e);
@ -1529,25 +1503,31 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
/**
* depend node is completed, but here need check the condition task branch is the next node
*/
private boolean dependTaskSuccess(String dependNodeName, String nextNodeName) {
if (dag.getNode(dependNodeName).isConditionsTask()) {
private boolean dependTaskSuccess(String dependNodeCode, String nextNodeCode) {
TaskNode dependentNode = dag.getNode(dependNodeCode);
if (dependentNode.isConditionsTask()) {
// condition task need check the branch to run
List<String> nextTaskList =
DagHelper.parseConditionTask(dependNodeName, skipTaskNodeMap, dag, getCompleteTaskInstanceMap());
if (!nextTaskList.contains(nextNodeName)) {
DagHelper.parseConditionTask(dependNodeCode, skipTaskNodeMap, dag, getCompleteTaskInstanceMap());
if (!nextTaskList.contains(nextNodeCode)) {
log.info(
"DependTask is a condition task, and its next condition branch does not hava current task, " +
"dependTaskCode: {}, currentTaskCode: {}",
dependNodeName, nextNodeName);
dependNodeCode, nextNodeCode);
return false;
}
} else {
long taskCode = Long.parseLong(dependNodeName);
Integer taskInstanceId = completeTaskMap.get(taskCode);
TaskExecutionStatus depTaskState = taskInstanceMap.get(taskInstanceId).getState();
return !depTaskState.isFailure();
return true;
}
return true;
if (dependentNode.isSwitchTask()) {
TaskInstance dependentTaskInstance = taskInstanceMap.get(validTaskMap.get(dependentNode.getCode()));
SwitchParameters switchParameters = dependentTaskInstance.getSwitchDependency();
return switchParameters.getDependTaskList().get(switchParameters.getResultConditionLocation()).getNextNode()
.contains(nextNodeCode);
}
long taskCode = Long.parseLong(dependNodeCode);
Integer taskInstanceId = completeTaskMap.get(taskCode);
TaskExecutionStatus depTaskState = taskInstanceMap.get(taskInstanceId).getState();
return !depTaskState.isFailure();
}
/**
@ -1612,7 +1592,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
return true;
}
if (processInstance.getFailureStrategy() == FailureStrategy.CONTINUE) {
return readyToSubmitTaskQueue.size() == 0 && activeTaskProcessorMaps.size() == 0
return readyToSubmitTaskQueue.size() == 0 && taskExecuteRunnableMap.size() == 0
&& waitToRetryTaskInstanceMap.size() == 0;
}
}
@ -1650,10 +1630,10 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
* @return ExecutionStatus
*/
private WorkflowExecutionStatus processReadyBlock() {
if (activeTaskProcessorMaps.size() > 0) {
for (ITaskProcessor taskProcessor : activeTaskProcessorMaps.values()) {
if (!TASK_TYPE_BLOCKING.equals(taskProcessor.getType())) {
taskProcessor.action(TaskAction.PAUSE);
if (taskExecuteRunnableMap.size() > 0) {
for (DefaultTaskExecuteRunnable taskExecuteRunnable : taskExecuteRunnableMap.values()) {
if (!TASK_TYPE_BLOCKING.equals(taskExecuteRunnable.getTaskInstance().getTaskType())) {
taskExecuteRunnable.pause();
}
}
}
@ -1673,7 +1653,7 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
private WorkflowExecutionStatus getProcessInstanceState(ProcessInstance instance) {
WorkflowExecutionStatus state = instance.getState();
if (activeTaskProcessorMaps.size() > 0 || hasRetryTaskInStandBy()) {
if (taskExecuteRunnableMap.size() > 0 || hasRetryTaskInStandBy()) {
// active task and retry task exists
WorkflowExecutionStatus executionStatus = runningState(state);
log.info("The workflowInstance has task running, the workflowInstance status is {}", executionStatus);
@ -1861,14 +1841,13 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
public void killAllTasks() {
log.info("kill called on process instance id: {}, num: {}",
processInstance.getId(),
activeTaskProcessorMaps.size());
taskExecuteRunnableMap.size());
if (readyToSubmitTaskQueue.size() > 0) {
readyToSubmitTaskQueue.clear();
}
for (long taskCode : activeTaskProcessorMaps.keySet()) {
ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskCode);
for (long taskCode : taskExecuteRunnableMap.keySet()) {
Integer taskInstanceId = validTaskMap.get(taskCode);
if (taskInstanceId == null || taskInstanceId.equals(0)) {
continue;
@ -1880,16 +1859,19 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
if (taskInstance == null || taskInstance.getState().isFinished()) {
continue;
}
taskProcessor.action(TaskAction.STOP);
if (taskProcessor.taskInstance().getState().isFinished()) {
TaskStateEvent taskStateEvent = TaskStateEvent.builder()
.processInstanceId(processInstance.getId())
.taskInstanceId(taskInstance.getId())
.status(taskProcessor.taskInstance().getState())
.type(StateEventType.TASK_STATE_CHANGE)
.build();
this.addStateEvent(taskStateEvent);
}
DefaultTaskExecuteRunnable defaultTaskExecuteRunnable = taskExecuteRunnableMap.get(taskCode);
CompletableFuture.runAsync(defaultTaskExecuteRunnable::kill)
.thenRun(() -> {
if (defaultTaskExecuteRunnable.getTaskInstance().getState().isFinished()) {
TaskStateEvent taskStateEvent = TaskStateEvent.builder()
.processInstanceId(processInstance.getId())
.taskInstanceId(taskInstance.getId())
.status(defaultTaskExecuteRunnable.getTaskInstance().getState())
.type(StateEventType.TASK_STATE_CHANGE)
.build();
this.addStateEvent(taskStateEvent);
}
});
}
}
}
@ -1902,12 +1884,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
* handling the list of tasks to be submitted
*/
public void submitStandByTask() throws StateEventHandleException {
int length = readyToSubmitTaskQueue.size();
for (int i = 0; i < length; i++) {
TaskInstance task = readyToSubmitTaskQueue.peek();
if (task == null) {
continue;
}
TaskInstance task;
while ((task = readyToSubmitTaskQueue.peek()) != null) {
// stop tasks which is retrying if forced success happens
if (task.taskCanRetry()) {
TaskInstance retryTask = taskInstanceDao.findTaskInstanceById(task.getId());
@ -1934,8 +1912,8 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
DependResult dependResult = getDependResultForTask(task);
if (DependResult.SUCCESS == dependResult) {
log.info("The dependResult of task {} is success, so ready to submit to execute", task.getName());
Optional<TaskInstance> taskInstanceOptional = submitTaskExec(task);
if (!taskInstanceOptional.isPresent()) {
if (!executeTask(task)) {
// todo: don't ste 0 here
task.setId(0);
this.taskFailedSubmit = true;
// Remove and add to complete map and error map
@ -1948,7 +1926,9 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
completeTaskMap.put(task.getTaskCode(), task.getId());
taskInstanceMap.put(task.getId(), task);
errorTaskMap.put(task.getTaskCode(), task.getId());
activeTaskProcessorMaps.remove(task.getTaskCode());
taskExecuteRunnableMap.remove(task.getTaskCode());
log.error("Task submitted failed, workflowInstanceId: {}, taskInstanceId: {}, taskCode: {}",
task.getProcessInstanceId(),
task.getId(),
@ -1960,13 +1940,13 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
// if the dependency fails, the current node is not submitted and the state changes to failure.
dependFailedTaskSet.add(task.getTaskCode());
removeTaskFromStandbyList(task);
log.info("Task dependent result is failed, taskInstanceId:{} depend result : {}", task.getId(),
log.info("Task dependent result is failed, taskInstanceName: {} depend result : {}", task.getName(),
dependResult);
} else if (DependResult.NON_EXEC == dependResult) {
// for some reasons(depend task pause/stop) this task would not be submit
removeTaskFromStandbyList(task);
log.info("Remove task due to depend result not executed, taskInstanceId:{} depend result : {}",
task.getId(), dependResult);
log.info("Remove task due to depend result not executed, taskInstanceName:{} depend result : {}",
task.getName(), dependResult);
}
}
}
@ -2078,22 +2058,12 @@ public class WorkflowExecuteRunnable implements Callable<WorkflowSubmitStatus> {
return false;
}
public void resubmit(long taskCode) throws Exception {
ITaskProcessor taskProcessor = activeTaskProcessorMaps.get(taskCode);
if (taskProcessor != null) {
taskProcessor.action(TaskAction.RESUBMIT);
log.debug("RESUBMIT: task code:{}", taskCode);
} else {
throw new Exception("resubmit error, taskProcessor is null, task code: " + taskCode);
}
}
public Map<Long, Integer> getCompleteTaskMap() {
return completeTaskMap;
}
public Map<Long, ITaskProcessor> getActiveTaskProcessMap() {
return activeTaskProcessorMaps;
public Map<Long, DefaultTaskExecuteRunnable> getTaskExecuteRunnableMap() {
return taskExecuteRunnableMap;
}
public Map<Long, TaskInstance> getWaitToRetryTaskInstanceMap() {

134
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/WorkflowExecuteRunnableFactory.java

@ -0,0 +1,134 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner;
import org.apache.dolphinscheduler.common.enums.SlotCheckState;
import org.apache.dolphinscheduler.dao.entity.Command;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.dao.repository.TaskDefinitionLogDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.exception.WorkflowCreateException;
import org.apache.dolphinscheduler.server.master.metrics.ProcessInstanceMetrics;
import org.apache.dolphinscheduler.server.master.registry.ServerNodeManager;
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient;
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnableFactory;
import org.apache.dolphinscheduler.service.alert.ProcessAlertManager;
import org.apache.dolphinscheduler.service.command.CommandService;
import org.apache.dolphinscheduler.service.expand.CuringParamsService;
import org.apache.dolphinscheduler.service.process.ProcessService;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class WorkflowExecuteRunnableFactory {
@Autowired
private ServerNodeManager serverNodeManager;
@Autowired
private CommandService commandService;
@Autowired
private ProcessService processService;
@Autowired
private ProcessInstanceDao processInstanceDao;
@Autowired
private MasterRpcClient masterRpcClient;
@Autowired
private ProcessAlertManager processAlertManager;
@Autowired
private StateWheelExecuteThread stateWheelExecuteThread;
@Autowired
private CuringParamsService curingGlobalParamsService;
@Autowired
private TaskInstanceDao taskInstanceDao;
@Autowired
private MasterConfig masterConfig;
@Autowired
private TaskDefinitionLogDao taskDefinitionLogDao;
@Autowired
private DefaultTaskExecuteRunnableFactory defaultTaskExecuteRunnableFactory;
public WorkflowExecuteRunnable createWorkflowExecuteRunnable(Command command) throws WorkflowCreateException {
try {
ProcessInstance workflowInstance = createWorkflowInstance(command);
return new WorkflowExecuteRunnable(workflowInstance,
commandService,
processService,
processInstanceDao,
masterRpcClient,
processAlertManager,
masterConfig,
stateWheelExecuteThread,
curingGlobalParamsService,
taskInstanceDao,
taskDefinitionLogDao,
defaultTaskExecuteRunnableFactory);
} catch (Exception ex) {
throw new WorkflowCreateException("Create workflow execute runnable failed", ex);
}
}
private ProcessInstance createWorkflowInstance(Command command) throws Exception {
long commandTransformStartTime = System.currentTimeMillis();
// Note: this check is not safe, the slot may change after command transform.
// We use the database transaction in `handleCommand` so that we can guarantee the command will
// always be executed
// by only one master
SlotCheckState slotCheckState = slotCheck(command);
if (slotCheckState.equals(SlotCheckState.CHANGE) || slotCheckState.equals(SlotCheckState.INJECT)) {
log.info("Master handle command {} skip, slot check state: {}", command.getId(), slotCheckState);
throw new RuntimeException("Slot check failed the current state: " + slotCheckState);
}
ProcessInstance processInstance = processService.handleCommand(masterConfig.getMasterAddress(), command);
log.info("Master handle command {} end, create process instance {}", command.getId(), processInstance.getId());
ProcessInstanceMetrics
.recordProcessInstanceGenerateTime(System.currentTimeMillis() - commandTransformStartTime);
return processInstance;
}
private SlotCheckState slotCheck(Command command) {
int slot = serverNodeManager.getSlot();
int masterSize = serverNodeManager.getMasterSize();
SlotCheckState state;
if (masterSize <= 0) {
state = SlotCheckState.CHANGE;
} else if (command.getId() % masterSize == slot) {
state = SlotCheckState.PASS;
} else {
state = SlotCheckState.INJECT;
}
return state;
}
}

50
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/MasterTaskDispatcher.java

@ -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.server.master.runner.dispatcher;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient;
import org.apache.dolphinscheduler.server.master.runner.BaseTaskDispatcher;
import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable;
import java.util.Optional;
import lombok.extern.slf4j.Slf4j;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class MasterTaskDispatcher extends BaseTaskDispatcher {
private final Optional<Host> masterTaskExecuteHost;
public MasterTaskDispatcher(TaskEventService taskEventService,
MasterConfig masterConfig,
MasterRpcClient masterRpcClient) {
super(taskEventService, masterConfig, masterRpcClient);
masterTaskExecuteHost = Optional.of(Host.of(masterConfig.getMasterAddress()));
}
@Override
protected Optional<Host> getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecutionContext) {
return masterTaskExecuteHost;
}
}

41
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatchFactory.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.server.master.runner.dispatcher;
import org.apache.dolphinscheduler.server.master.utils.TaskUtils;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class TaskDispatchFactory {
@Autowired
private MasterTaskDispatcher masterTaskDispatcher;
@Autowired
private WorkerTaskDispatcher workerTaskDispatcher;
public TaskDispatcher getTaskDispatcher(String taskType) {
return TaskUtils.isMasterTask(taskType) ? masterTaskDispatcher : workerTaskDispatcher;
}
}

31
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/TaskDispatcher.java

@ -0,0 +1,31 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.dispatcher;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
import org.apache.dolphinscheduler.server.master.exception.TaskDispatchException;
import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable;
/**
* Used to do task dispatcher.
*/
public interface TaskDispatcher {
void dispatchTask(TaskExecuteRunnable taskExecuteRunnable) throws TaskDispatchException, WorkerGroupNotFoundException;
}

57
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/dispatcher/WorkerTaskDispatcher.java

@ -0,0 +1,57 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.dispatcher;
import static com.google.common.base.Preconditions.checkNotNull;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.dispatch.exceptions.WorkerGroupNotFoundException;
import org.apache.dolphinscheduler.server.master.dispatch.host.HostManager;
import org.apache.dolphinscheduler.server.master.processor.queue.TaskEventService;
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient;
import org.apache.dolphinscheduler.server.master.runner.BaseTaskDispatcher;
import org.apache.dolphinscheduler.server.master.runner.execute.TaskExecuteRunnable;
import java.util.Optional;
import lombok.extern.slf4j.Slf4j;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class WorkerTaskDispatcher extends BaseTaskDispatcher {
private final HostManager hostManager;
public WorkerTaskDispatcher(TaskEventService taskEventService,
MasterConfig masterConfig,
MasterRpcClient masterRpcClient,
HostManager hostManager) {
super(taskEventService, masterConfig, masterRpcClient);
this.hostManager = checkNotNull(hostManager);
}
@Override
protected Optional<Host> getTaskInstanceDispatchHost(TaskExecuteRunnable taskExecuteRunnable) throws WorkerGroupNotFoundException {
String workerGroup = taskExecuteRunnable.getTaskExecutionContext().getWorkerGroup();
return hostManager.select(workerGroup);
}
}

60
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterDelayTaskExecuteRunnable.java

@ -0,0 +1,60 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager;
import org.apache.dolphinscheduler.server.master.runner.task.IAsyncLogicTask;
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public class AsyncMasterDelayTaskExecuteRunnable extends MasterDelayTaskExecuteRunnable {
private final AsyncMasterTaskDelayQueue asyncMasterTaskDelayQueue;
public AsyncMasterDelayTaskExecuteRunnable(TaskExecutionContext taskExecutionContext,
LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder,
MasterMessageSenderManager masterMessageSenderManager,
AsyncMasterTaskDelayQueue asyncTaskDelayQueue) {
super(taskExecutionContext, logicTaskPluginFactoryBuilder, masterMessageSenderManager);
this.asyncMasterTaskDelayQueue = asyncTaskDelayQueue;
}
@Override
protected void executeTask() throws MasterTaskExecuteException {
if (logicTask == null) {
throw new MasterTaskExecuteException("The task plugin instance is null");
}
final IAsyncLogicTask iAsyncLogicTask = (IAsyncLogicTask) logicTask;
// we execute the handle method here, but for async task, this method will not block
// submit the task to async task queue
final AsyncTaskExecutionContext asyncTaskExecutionContext = new AsyncTaskExecutionContext(
taskExecutionContext,
iAsyncLogicTask.getAsyncTaskExecuteFunction(),
new AsyncTaskCallbackFunctionImpl(this));
asyncMasterTaskDelayQueue.addAsyncTask(asyncTaskExecutionContext);
}
@Override
protected void afterExecute() {
// Do nothing, async task is not finished yet
}
}

49
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterDelayTaskExecuteRunnableFactory.java

@ -0,0 +1,49 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager;
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Component
public class AsyncMasterDelayTaskExecuteRunnableFactory
implements
MasterDelayTaskExecuteRunnableFactory<AsyncMasterDelayTaskExecuteRunnable> {
@Autowired
private LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder;
@Autowired
private MasterMessageSenderManager masterMessageSenderManager;
@Autowired
private AsyncMasterTaskDelayQueue asyncTaskDelayQueue;
@Override
public AsyncMasterDelayTaskExecuteRunnable createWorkerTaskExecuteRunnable(TaskExecutionContext taskExecutionContext) {
return new AsyncMasterDelayTaskExecuteRunnable(taskExecutionContext,
logicTaskPluginFactoryBuilder,
masterMessageSenderManager,
asyncTaskDelayQueue);
}
}

46
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueue.java

@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import java.util.concurrent.DelayQueue;
import javax.annotation.Nullable;
import lombok.NonNull;
import org.springframework.stereotype.Component;
@Component
public class AsyncMasterTaskDelayQueue {
private final DelayQueue<AsyncTaskExecutionContext> asyncTaskCheckDelayQueue = new DelayQueue<>();
public void addAsyncTask(@NonNull AsyncTaskExecutionContext asyncTaskExecutionContext) {
asyncTaskExecutionContext.refreshStartTime();
asyncTaskCheckDelayQueue.add(asyncTaskExecutionContext);
}
public @Nullable AsyncTaskExecutionContext pollAsyncTask() throws InterruptedException {
return asyncTaskCheckDelayQueue.take();
}
public int getAsyncTaskRunningNum() {
return asyncTaskCheckDelayQueue.size();
}
}

132
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncMasterTaskDelayQueueLooper.java

@ -0,0 +1,132 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.common.thread.BaseDaemonThread;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class AsyncMasterTaskDelayQueueLooper extends BaseDaemonThread implements AutoCloseable {
@Autowired
private AsyncMasterTaskDelayQueue asyncMasterTaskDelayQueue;
@Autowired
private MasterConfig masterConfig;
private static final AtomicBoolean RUNNING_FLAG = new AtomicBoolean(false);
private ExecutorService asyncTaskStateCheckThreadPool;
public AsyncMasterTaskDelayQueueLooper() {
super("AsyncMasterTaskDelayQueueLooper");
}
@Override
public synchronized void start() {
if (!RUNNING_FLAG.compareAndSet(false, true)) {
log.info("The AsyncMasterTaskDelayQueueLooper has already been started, will not start again");
return;
}
log.info("AsyncMasterTaskDelayQueueLooper starting...");
super.start();
log.info("AsyncMasterTaskDelayQueueLooper started...");
}
@Override
public void run() {
asyncTaskStateCheckThreadPool = ThreadUtils.newDaemonFixedThreadExecutor("AsyncTaskStateCheckThreadPool",
masterConfig.getMasterAsyncTaskStateCheckThreadPoolSize());
while (RUNNING_FLAG.get()) {
AsyncTaskExecutionContext asyncTaskExecutionContext;
try {
asyncTaskExecutionContext = asyncMasterTaskDelayQueue.pollAsyncTask();
} catch (InterruptedException e) {
log.error("AsyncConditionTaskLooper has been interrupted, will break this loop", e);
Thread.currentThread().interrupt();
break;
}
final TaskExecutionContext taskExecutionContext = asyncTaskExecutionContext.getTaskExecutionContext();
try (
LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC(
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId());
LogUtils.MDCAutoClosableContext mdcAutoClosableContext1 =
LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath())) {
if (MasterTaskExecutionContextHolder
.getTaskExecutionContext(taskExecutionContext.getTaskInstanceId()) == null) {
log.warn(
"Cannot find the taskInstance from TaskExecutionContextCacheManager, the task may already been killed, will stop the async master task");
continue;
}
asyncTaskStateCheckThreadPool.submit(() -> {
final AsyncTaskExecuteFunction asyncTaskExecuteFunction =
asyncTaskExecutionContext.getAsyncTaskExecuteFunction();
final AsyncTaskCallbackFunction asyncTaskCallbackFunction =
asyncTaskExecutionContext.getAsyncTaskCallbackFunction();
try (
LogUtils.MDCAutoClosableContext mdcAutoClosableContext2 =
LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath())) {
AsyncTaskExecuteFunction.AsyncTaskExecutionStatus asyncTaskExecutionStatus =
asyncTaskExecuteFunction.getAsyncTaskExecutionStatus();
switch (asyncTaskExecutionStatus) {
case RUNNING:
// If the task status is running, means the task real status is not finished. We will
// put it back to the queue to get the status again.
asyncMasterTaskDelayQueue.addAsyncTask(asyncTaskExecutionContext);
break;
case SUCCESS:
asyncTaskCallbackFunction.executeSuccess();
break;
case FAILED:
asyncTaskCallbackFunction.executeFailed();
break;
}
} catch (Exception ex) {
asyncTaskCallbackFunction.executeThrowing(ex);
}
});
}
}
log.info("AsyncMasterTaskDelayQueueLooper closed...");
}
@Override
public void close() throws Exception {
if (!RUNNING_FLAG.compareAndSet(true, false)) {
log.warn("The AsyncMasterTaskDelayQueueLooper is not started, will not close");
return;
}
log.info("AsyncMasterTaskDelayQueueLooper closing...");
asyncTaskStateCheckThreadPool.shutdown();
log.info("AsyncMasterTaskDelayQueueLooper closed...");
}
}

30
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunction.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
public interface AsyncTaskCallbackFunction {
void executeSuccess() throws MasterTaskExecuteException;
void executeFailed() throws MasterTaskExecuteException;
void executeThrowing(Throwable throwable);
}

66
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskCallbackFunctionImpl.java

@ -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.server.master.runner.execute;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContextCacheManager;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.log.TaskInstanceLogHeader;
import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public class AsyncTaskCallbackFunctionImpl implements AsyncTaskCallbackFunction {
private final AsyncMasterDelayTaskExecuteRunnable asyncMasterDelayTaskExecuteRunnable;
public AsyncTaskCallbackFunctionImpl(@NonNull AsyncMasterDelayTaskExecuteRunnable asyncMasterDelayTaskExecuteRunnable) {
this.asyncMasterDelayTaskExecuteRunnable = asyncMasterDelayTaskExecuteRunnable;
}
@Override
public void executeSuccess() {
asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext()
.setCurrentExecutionStatus(TaskExecutionStatus.SUCCESS);
executeFinished();
}
@Override
public void executeFailed() {
asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext()
.setCurrentExecutionStatus(TaskExecutionStatus.FAILURE);
executeFinished();
}
@Override
public void executeThrowing(Throwable throwable) {
asyncMasterDelayTaskExecuteRunnable.afterThrowing(throwable);
}
private void executeFinished() {
TaskInstanceLogHeader.printFinalizeTaskHeader();
TaskExecutionContextCacheManager.removeByTaskInstanceId(
asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext().getTaskInstanceId());
log.info("Task execute finished, removed the TaskExecutionContext");
asyncMasterDelayTaskExecuteRunnable.sendTaskResult();
log.info(
"Execute task finished, will send the task execute result to master, the current task execute result is {}",
asyncMasterDelayTaskExecuteRunnable.getTaskExecutionContext().getCurrentExecutionStatus().name());
asyncMasterDelayTaskExecuteRunnable.closeLogAppender();
}
}

43
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecuteFunction.java

@ -0,0 +1,43 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import java.time.Duration;
import lombok.NonNull;
public interface AsyncTaskExecuteFunction {
@NonNull
AsyncTaskExecutionStatus getAsyncTaskExecutionStatus();
@NonNull
Duration getAsyncTaskStateCheckInterval();
enum AsyncTaskExecutionStatus {
RUNNING,
SUCCESS,
FAILED,
;
}
}

84
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/AsyncTaskExecutionContext.java

@ -0,0 +1,84 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy;
import java.util.concurrent.Delayed;
import java.util.concurrent.TimeUnit;
import lombok.Data;
import lombok.NonNull;
@Data
public class AsyncTaskExecutionContext implements Delayed {
private final TaskExecutionContext taskExecutionContext;
private final AsyncTaskExecuteFunction asyncTaskExecuteFunction;
private final AsyncTaskCallbackFunction asyncTaskCallbackFunction;
private long currentStartTime;
private int executeTimes;
private final long executeInterval;
private long timeout;
public AsyncTaskExecutionContext(@NonNull TaskExecutionContext taskExecutionContext,
@NonNull AsyncTaskExecuteFunction asyncTaskExecuteFunction,
@NonNull AsyncTaskCallbackFunction asyncTaskCallbackFunction) {
this.taskExecutionContext = taskExecutionContext;
this.asyncTaskExecuteFunction = asyncTaskExecuteFunction;
this.asyncTaskCallbackFunction = asyncTaskCallbackFunction;
this.currentStartTime = 0;
this.executeTimes = 0;
if (TaskTimeoutStrategy.FAILED.equals(taskExecutionContext.getTaskTimeoutStrategy())
|| TaskTimeoutStrategy.WARNFAILED.equals(taskExecutionContext.getTaskTimeoutStrategy())) {
// will timeout
this.timeout = taskExecutionContext.getStartTime()
+ TimeUnit.SECONDS.toMillis(taskExecutionContext.getTaskTimeout());
} else {
this.timeout = TimeUnit.SECONDS.toMillis(Integer.MAX_VALUE);
}
this.executeInterval = Math.max(asyncTaskExecuteFunction.getAsyncTaskStateCheckInterval().toMillis(), 1000L);
}
public void refreshStartTime() {
if (executeTimes == 0) {
// The first time doesn't have delay
executeTimes++;
} else {
currentStartTime = System.currentTimeMillis();
}
}
@Override
public long getDelay(TimeUnit unit) {
long nextExecuteTimeDelay = Math.min(currentStartTime + executeInterval, timeout) - System.currentTimeMillis();
return unit.convert(nextExecuteTimeDelay, TimeUnit.MILLISECONDS);
}
@Override
public int compareTo(Delayed o) {
if (o == null) {
return 1;
}
return Long.compare(this.getDelay(TimeUnit.SECONDS), o.getDelay(TimeUnit.SECONDS));
}
}

58
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnable.java

@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import static com.google.common.base.Preconditions.checkNotNull;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.runner.operator.TaskOperatorManager;
public class DefaultTaskExecuteRunnable extends PriorityTaskExecuteRunnable {
private final TaskOperatorManager taskOperatorManager;
public DefaultTaskExecuteRunnable(ProcessInstance workflowInstance,
TaskInstance taskInstance,
TaskExecutionContext taskExecutionContext,
TaskOperatorManager taskOperatorManager) {
super(workflowInstance, taskInstance, taskExecutionContext);
this.taskOperatorManager = checkNotNull(taskOperatorManager);
}
@Override
public void dispatch() {
taskOperatorManager.getTaskDispatchOperator().handle(this);
}
@Override
public void kill() {
taskOperatorManager.getTaskKillOperator().handle(this);
}
@Override
public void pause() {
taskOperatorManager.getTaskPauseOperator().handle(this);
}
@Override
public void timeout() {
taskOperatorManager.getTaskTimeoutOperator().handle(this);
}
}

59
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/DefaultTaskExecuteRunnableFactory.java

@ -0,0 +1,59 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.server.master.cache.ProcessInstanceExecCacheManager;
import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException;
import org.apache.dolphinscheduler.server.master.exception.TaskExecutionContextCreateException;
import org.apache.dolphinscheduler.server.master.runner.WorkflowExecuteRunnable;
import org.apache.dolphinscheduler.server.master.runner.operator.TaskOperatorManager;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class DefaultTaskExecuteRunnableFactory implements TaskExecuteRunnableFactory<DefaultTaskExecuteRunnable> {
@Autowired
private ProcessInstanceExecCacheManager processInstanceExecCacheManager;
@Autowired
private TaskExecutionContextFactory taskExecutionContextFactory;
@Autowired
private TaskOperatorManager taskOperatorManager;
@Override
public DefaultTaskExecuteRunnable createTaskExecuteRunnable(TaskInstance taskInstance) throws TaskExecuteRunnableCreateException {
WorkflowExecuteRunnable workflowExecuteRunnable =
processInstanceExecCacheManager.getByProcessInstanceId(taskInstance.getProcessInstanceId());
try {
return new DefaultTaskExecuteRunnable(
workflowExecuteRunnable.getProcessInstance(),
taskInstance,
taskExecutionContextFactory.createTaskExecutionContext(taskInstance),
taskOperatorManager);
} catch (TaskExecutionContextCreateException ex) {
throw new TaskExecuteRunnableCreateException("Create DefaultTaskExecuteRunnable failed", ex);
}
}
}

68
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterDelayTaskExecuteRunnable.java

@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager;
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder;
import java.util.concurrent.Delayed;
import java.util.concurrent.TimeUnit;
public abstract class MasterDelayTaskExecuteRunnable extends MasterTaskExecuteRunnable implements Delayed {
public MasterDelayTaskExecuteRunnable(TaskExecutionContext taskExecutionContext,
LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder,
MasterMessageSenderManager masterMessageSenderManager) {
super(taskExecutionContext, logicTaskPluginFactoryBuilder, masterMessageSenderManager);
}
@Override
public boolean equals(Object obj) {
if (!(obj instanceof MasterDelayTaskExecuteRunnable)) {
return false;
}
MasterDelayTaskExecuteRunnable other = (MasterDelayTaskExecuteRunnable) obj;
return other.getTaskExecutionContext().getTaskInstanceId() == this.getTaskExecutionContext()
.getTaskInstanceId();
}
@Override
public int hashCode() {
return this.getTaskExecutionContext().getTaskInstanceId();
}
@Override
public long getDelay(TimeUnit unit) {
TaskExecutionContext taskExecutionContext = getTaskExecutionContext();
return unit.convert(
DateUtils.getRemainTime(
taskExecutionContext.getFirstSubmitTime(), taskExecutionContext.getDelayTime() * 60L),
TimeUnit.SECONDS);
}
@Override
public int compareTo(Delayed o) {
if (o == null) {
return 1;
}
return Long.compare(this.getDelay(TimeUnit.MILLISECONDS), o.getDelay(TimeUnit.MILLISECONDS));
}
}

26
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterDelayTaskExecuteRunnableFactory.java

@ -0,0 +1,26 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
public interface MasterDelayTaskExecuteRunnableFactory<T extends MasterDelayTaskExecuteRunnable> {
T createWorkerTaskExecuteRunnable(TaskExecutionContext taskExecutionContext);
}

173
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecuteRunnable.java

@ -0,0 +1,173 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import static ch.qos.logback.classic.ClassicConstants.FINALIZE_SESSION_MARKER;
import org.apache.dolphinscheduler.common.log.remote.RemoteLogUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContextCacheManager;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.log.TaskInstanceLogHeader;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.remote.exceptions.RemotingException;
import org.apache.dolphinscheduler.server.master.exception.LogicTaskFactoryNotFoundException;
import org.apache.dolphinscheduler.server.master.exception.LogicTaskInitializeException;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager;
import org.apache.dolphinscheduler.server.master.runner.task.ILogicTask;
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public abstract class MasterTaskExecuteRunnable implements Runnable {
protected final TaskExecutionContext taskExecutionContext;
protected final LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder;
protected final MasterMessageSenderManager masterMessageSenderManager;
protected ILogicTask logicTask;
public MasterTaskExecuteRunnable(TaskExecutionContext taskExecutionContext,
LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder,
MasterMessageSenderManager masterMessageSenderManager) {
this.taskExecutionContext = taskExecutionContext;
this.logicTaskPluginFactoryBuilder = logicTaskPluginFactoryBuilder;
this.masterMessageSenderManager = masterMessageSenderManager;
}
protected abstract void executeTask() throws MasterTaskExecuteException;
protected abstract void afterExecute() throws MasterTaskExecuteException;
protected void afterThrowing(Throwable throwable) {
try {
cancelTask();
log.info("Get a exception when execute the task, canceled the task");
} catch (Exception e) {
log.error("Cancel task failed,", e);
}
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.FAILURE);
sendTaskResult();
log.info(
"Get a exception when execute the task, sent the task execute result to master, the current task execute result is {}",
taskExecutionContext.getCurrentExecutionStatus());
TaskExecutionContextCacheManager.removeByTaskInstanceId(taskExecutionContext.getTaskInstanceId());
log.info("Get a exception when execute the task, removed the TaskExecutionContext");
}
public void cancelTask() throws MasterTaskExecuteException {
if (logicTask != null) {
logicTask.kill();
}
}
public void pauseTask() throws MasterTaskExecuteException {
if (logicTask != null) {
logicTask.pause();
}
}
public TaskExecutionContext getTaskExecutionContext() {
return taskExecutionContext;
}
@Override
public void run() {
try (
final LogUtils.MDCAutoClosableContext mdcAutoClosableContext = LogUtils.setWorkflowAndTaskInstanceIDMDC(
taskExecutionContext.getProcessInstanceId(), taskExecutionContext.getTaskInstanceId());
final LogUtils.MDCAutoClosableContext mdcAutoClosableContext1 =
LogUtils.setTaskInstanceLogFullPathMDC(taskExecutionContext.getLogPath())) {
TaskInstanceLogHeader.printInitializeTaskContextHeader();
initializeTask();
TaskInstanceLogHeader.printLoadTaskInstancePluginHeader();
beforeExecute();
TaskInstanceLogHeader.printExecuteTaskHeader();
executeTask();
afterExecute();
} catch (Throwable ex) {
log.error("Task execute failed, due to meet an exception", ex);
afterThrowing(ex);
closeLogAppender();
}
}
protected void initializeTask() {
log.info("Begin to initialize task");
long taskStartTime = System.currentTimeMillis();
taskExecutionContext.setStartTime(taskStartTime);
log.info("Set task startTime: {}", taskStartTime);
String taskAppId = String.format("%s_%s", taskExecutionContext.getProcessInstanceId(),
taskExecutionContext.getTaskInstanceId());
taskExecutionContext.setTaskAppId(taskAppId);
log.info("Set task appId: {}", taskAppId);
log.info("End initialize task {}", JSONUtils.toPrettyJsonString(taskExecutionContext));
}
protected void beforeExecute() throws LogicTaskFactoryNotFoundException, LogicTaskInitializeException, RemotingException {
taskExecutionContext.setCurrentExecutionStatus(TaskExecutionStatus.RUNNING_EXECUTION);
masterMessageSenderManager.getMasterTaskExecuteRunningMessageSender().sendMessage(taskExecutionContext);
log.info("Send task status {} to master {}", taskExecutionContext.getCurrentExecutionStatus().name(),
taskExecutionContext.getWorkflowInstanceHost());
logicTask = logicTaskPluginFactoryBuilder.createILogicTaskPluginFactory(taskExecutionContext.getTaskType())
.createLogicTask(taskExecutionContext);
log.info("Initialized task plugin instance: {} successfully", taskExecutionContext.getTaskType());
logicTask.getTaskParameters().setVarPool(taskExecutionContext.getVarPool());
log.info("Initialize taskVarPool: {} successfully", taskExecutionContext.getVarPool());
}
protected void closeLogAppender() {
try {
if (RemoteLogUtils.isRemoteLoggingEnable()) {
RemoteLogUtils.sendRemoteLog(taskExecutionContext.getLogPath());
log.info("Send task log {} to remote storage successfully", taskExecutionContext.getLogPath());
}
} catch (Exception ex) {
log.error("Send task log {} to remote storage failed", taskExecutionContext.getLogPath(), ex);
} finally {
log.info(FINALIZE_SESSION_MARKER, FINALIZE_SESSION_MARKER.toString());
}
}
protected void sendTaskResult() {
try {
taskExecutionContext.setEndTime(System.currentTimeMillis());
taskExecutionContext.setVarPool(JSONUtils.toJsonString(logicTask.getTaskParameters().getVarPool()));
masterMessageSenderManager.getMasterTaskExecuteResultMessageSender().sendMessage(taskExecutionContext);
log.info("Send task status: {} to master: {} successfully",
taskExecutionContext.getCurrentExecutionStatus().name(),
taskExecutionContext.getWorkflowInstanceHost());
} catch (Exception ex) {
log.error("Send task status: {} to master: {} failed",
taskExecutionContext.getCurrentExecutionStatus().name(),
taskExecutionContext.getWorkflowInstanceHost(), ex);
}
}
}

49
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecuteRunnableFactoryBuilder.java

@ -0,0 +1,49 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.server.master.runner.task.dependent.DependentLogicTask;
import org.apache.dolphinscheduler.server.master.runner.task.subworkflow.SubWorkflowLogicTask;
import java.util.Set;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import com.google.common.collect.Sets;
@Component
public class MasterTaskExecuteRunnableFactoryBuilder {
@Autowired
private AsyncMasterDelayTaskExecuteRunnableFactory asyncMasterDelayTaskExecuteRunnableFactory;
@Autowired
private SyncMasterDelayTaskExecuteRunnableFactory syncMasterDelayTaskExecuteRunnableFactory;
private static final Set<String> ASYNC_TASK_TYPE = Sets.newHashSet(
DependentLogicTask.TASK_TYPE,
SubWorkflowLogicTask.TASK_TYPE);
public MasterDelayTaskExecuteRunnableFactory<? extends MasterDelayTaskExecuteRunnable> createWorkerDelayTaskExecuteRunnableFactory(String taskType) {
if (ASYNC_TASK_TYPE.contains(taskType)) {
return asyncMasterDelayTaskExecuteRunnableFactory;
}
return syncMasterDelayTaskExecuteRunnableFactory;
}
}

47
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/MasterTaskExecutionContextHolder.java

@ -0,0 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public class MasterTaskExecutionContextHolder {
private static final Map<Integer, TaskExecutionContext> TASK_EXECUTION_CONTEXT_MAP = new ConcurrentHashMap<>();
public static void putTaskExecutionContext(TaskExecutionContext taskExecutionContext) {
if (TASK_EXECUTION_CONTEXT_MAP.containsKey(taskExecutionContext.getTaskInstanceId())) {
log.error("The TaskExecutionContext {} already exists in the MasterTaskExecutionContextHolder",
taskExecutionContext);
}
TASK_EXECUTION_CONTEXT_MAP.put(taskExecutionContext.getTaskInstanceId(), taskExecutionContext);
}
public static TaskExecutionContext getTaskExecutionContext(int taskInstanceId) {
return TASK_EXECUTION_CONTEXT_MAP.get(taskInstanceId);
}
public static void removeTaskExecutionContext(int taskInstanceId) {
TASK_EXECUTION_CONTEXT_MAP.remove(taskInstanceId);
}
}

97
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/PriorityTaskExecuteRunnable.java

@ -0,0 +1,97 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import static com.google.common.base.Preconditions.checkNotNull;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.jetbrains.annotations.NotNull;
public abstract class PriorityTaskExecuteRunnable implements TaskExecuteRunnable, Comparable<TaskExecuteRunnable> {
private final ProcessInstance workflowInstance;
private final TaskInstance taskInstance;
private final TaskExecutionContext taskExecutionContext;
public PriorityTaskExecuteRunnable(ProcessInstance workflowInstance,
TaskInstance taskInstance,
TaskExecutionContext taskExecutionContext) {
this.taskInstance = checkNotNull(taskInstance);
this.workflowInstance = checkNotNull(workflowInstance);
this.taskExecutionContext = checkNotNull(taskExecutionContext);
}
@Override
public ProcessInstance getWorkflowInstance() {
return workflowInstance;
}
@Override
public TaskInstance getTaskInstance() {
return taskInstance;
}
@Override
public TaskExecutionContext getTaskExecutionContext() {
return taskExecutionContext;
}
@Override
public int compareTo(@NotNull TaskExecuteRunnable other) {
int workflowInstancePriorityCompareResult = workflowInstance.getProcessInstancePriority().getCode()
- other.getWorkflowInstance().getProcessInstancePriority().getCode();
if (workflowInstancePriorityCompareResult != 0) {
return workflowInstancePriorityCompareResult;
}
int workflowInstanceIdCompareResult = workflowInstance.getId() - other.getWorkflowInstance().getId();
if (workflowInstanceIdCompareResult != 0) {
return workflowInstanceIdCompareResult;
}
int taskInstancePriorityCompareResult = taskInstance.getTaskInstancePriority().getCode()
- other.getTaskInstance().getTaskInstancePriority().getCode();
if (taskInstancePriorityCompareResult > 0) {
return taskInstancePriorityCompareResult;
}
// larger number, higher priority
int taskGroupPriorityCompareResult =
taskInstance.getTaskGroupPriority() - other.getTaskInstance().getTaskGroupPriority();
if (taskGroupPriorityCompareResult != 0) {
return -taskGroupPriorityCompareResult;
}
// The task instance shouldn't be equals
return taskInstance.getId() - other.getTaskInstance().getId();
}
@Override
public boolean equals(Object obj) {
if (obj instanceof PriorityTaskExecuteRunnable) {
PriorityTaskExecuteRunnable other = (PriorityTaskExecuteRunnable) obj;
return compareTo(other) == 0;
}
return false;
}
@Override
public int hashCode() {
return taskInstance.getId();
}
}

60
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterDelayTaskExecuteRunnable.java

@ -0,0 +1,60 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.log.TaskInstanceLogHeader;
import org.apache.dolphinscheduler.server.master.exception.MasterTaskExecuteException;
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager;
import org.apache.dolphinscheduler.server.master.runner.task.ISyncLogicTask;
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder;
import lombok.extern.slf4j.Slf4j;
@Slf4j
public class SyncMasterDelayTaskExecuteRunnable extends MasterDelayTaskExecuteRunnable {
public SyncMasterDelayTaskExecuteRunnable(TaskExecutionContext taskExecutionContext,
LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder,
MasterMessageSenderManager masterMessageSenderManager) {
super(taskExecutionContext, logicTaskPluginFactoryBuilder, masterMessageSenderManager);
}
@Override
protected void executeTask() throws MasterTaskExecuteException {
if (logicTask == null) {
throw new MasterTaskExecuteException("The task plugin instance is null");
}
ISyncLogicTask iSyncLogicTask = (ISyncLogicTask) logicTask;
iSyncLogicTask.handle();
}
protected void afterExecute() throws MasterTaskExecuteException {
TaskInstanceLogHeader.printFinalizeTaskHeader();
// todo: move this to constructor to avoid check every time
if (logicTask == null) {
throw new MasterTaskExecuteException("The current task instance is null");
}
sendTaskResult();
log.info(
"Execute task finished, will send the task execute result to master, the current task execute result is {}",
taskExecutionContext.getCurrentExecutionStatus().name());
closeLogAppender();
}
}

45
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/SyncMasterDelayTaskExecuteRunnableFactory.java

@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.server.master.runner.message.MasterMessageSenderManager;
import org.apache.dolphinscheduler.server.master.runner.task.LogicTaskPluginFactoryBuilder;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class SyncMasterDelayTaskExecuteRunnableFactory
implements
MasterDelayTaskExecuteRunnableFactory<SyncMasterDelayTaskExecuteRunnable> {
@Autowired
private LogicTaskPluginFactoryBuilder logicTaskPluginFactoryBuilder;
@Autowired
private MasterMessageSenderManager masterMessageSenderManager;
@Override
public SyncMasterDelayTaskExecuteRunnable createWorkerTaskExecuteRunnable(TaskExecutionContext taskExecutionContext) {
return new SyncMasterDelayTaskExecuteRunnable(taskExecutionContext, logicTaskPluginFactoryBuilder,
masterMessageSenderManager);
}
}

43
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecuteRunnable.java

@ -0,0 +1,43 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
/**
* This interface is used to define a task which is executing.
* todo: split to MasterTaskExecuteRunnable and WorkerTaskExecuteRunnable
*/
public interface TaskExecuteRunnable extends Comparable<TaskExecuteRunnable> {
void dispatch();
void kill();
void pause();
void timeout();
ProcessInstance getWorkflowInstance();
TaskInstance getTaskInstance();
TaskExecutionContext getTaskExecutionContext();
}

27
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecuteRunnableFactory.java

@ -0,0 +1,27 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.execute;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.server.master.exception.TaskExecuteRunnableCreateException;
public interface TaskExecuteRunnableFactory<T extends TaskExecuteRunnable> {
T createTaskExecuteRunnable(TaskInstance taskInstance) throws TaskExecuteRunnableCreateException;
}

493
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/task/BaseTaskProcessor.java → dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/execute/TaskExecutionContextFactory.java

@ -15,7 +15,7 @@
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.task;
package org.apache.dolphinscheduler.server.master.runner.execute;
import static org.apache.dolphinscheduler.common.constants.Constants.ADDRESS;
import static org.apache.dolphinscheduler.common.constants.Constants.DATABASE;
@ -46,15 +46,10 @@ import org.apache.dolphinscheduler.dao.entity.DqRuleInputEntry;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.entity.UdfFunc;
import org.apache.dolphinscheduler.dao.repository.ProcessInstanceDao;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.plugin.storage.api.StorageOperate;
import org.apache.dolphinscheduler.plugin.task.api.DataQualityTaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.K8sTaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.TaskChannel;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.plugin.task.api.enums.dp.ConnectorType;
import org.apache.dolphinscheduler.plugin.task.api.enums.dp.ExecuteSqlType;
import org.apache.dolphinscheduler.plugin.task.api.model.JdbcInfo;
@ -69,17 +64,15 @@ import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.DataSourc
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.ResourceParametersHelper;
import org.apache.dolphinscheduler.plugin.task.api.parameters.resource.UdfFuncParameters;
import org.apache.dolphinscheduler.plugin.task.api.utils.JdbcUrlParser;
import org.apache.dolphinscheduler.plugin.task.api.utils.LogUtils;
import org.apache.dolphinscheduler.plugin.task.api.utils.MapUtils;
import org.apache.dolphinscheduler.plugin.task.spark.SparkParameters;
import org.apache.dolphinscheduler.server.master.builder.TaskExecutionContextBuilder;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.server.master.exception.TaskExecutionContextCreateException;
import org.apache.dolphinscheduler.service.expand.CuringParamsService;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.spi.enums.DbType;
import org.apache.dolphinscheduler.spi.enums.ResourceType;
import org.apache.dolphinscheduler.spi.plugin.SPIIdentify;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
@ -89,252 +82,60 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Properties;
import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import com.zaxxer.hikari.HikariDataSource;
public abstract class BaseTaskProcessor implements ITaskProcessor {
@Slf4j
@Component
public class TaskExecutionContextFactory {
protected final Logger log =
LoggerFactory.getLogger(BaseTaskProcessor.class);
@Autowired
private ProcessService processService;
private String tenantCode;
@Autowired
private TaskPluginManager taskPluginManager;
protected boolean killed = false;
@Autowired
private CuringParamsService curingParamsService;
protected boolean paused = false;
@Autowired
private MasterConfig masterConfig;
protected boolean timeout = false;
@Autowired
private HikariDataSource hikariDataSource;
protected TaskInstance taskInstance = null;
protected ProcessInstance processInstance;
protected int maxRetryTimes;
protected long commitInterval;
protected ProcessService processService;
protected TaskInstanceDao taskInstanceDao;
protected ProcessInstanceDao processInstanceDao;
protected StorageOperate storageOperate;
protected MasterConfig masterConfig;
protected TaskPluginManager taskPluginManager;
protected CuringParamsService curingParamsService;
protected String threadLoggerInfoName;
@Override
public void init(@NonNull TaskInstance taskInstance, @NonNull ProcessInstance processInstance) {
processService = SpringApplicationContext.getBean(ProcessService.class);
processInstanceDao = SpringApplicationContext.getBean(ProcessInstanceDao.class);
masterConfig = SpringApplicationContext.getBean(MasterConfig.class);
taskPluginManager = SpringApplicationContext.getBean(TaskPluginManager.class);
curingParamsService = SpringApplicationContext.getBean(CuringParamsService.class);
taskInstanceDao = SpringApplicationContext.getBean(TaskInstanceDao.class);
storageOperate = SpringApplicationContext.getBean(StorageOperate.class, null);
this.taskInstance = taskInstance;
this.processInstance = processInstance;
this.maxRetryTimes = masterConfig.getTaskCommitRetryTimes();
this.commitInterval = masterConfig.getTaskCommitInterval().toMillis();
this.tenantCode = getTenantCode();
}
protected javax.sql.DataSource defaultDataSource =
SpringApplicationContext.getBean(javax.sql.DataSource.class);
/**
* pause task, common tasks donot need this.
*/
protected abstract boolean pauseTask();
/**
* kill task, all tasks need to realize this function
*/
protected abstract boolean killTask();
/**
* task timeout process
*/
protected abstract boolean taskTimeout();
/**
* submit task
*/
protected abstract boolean submitTask();
/*
* resubmit task
*/
protected abstract boolean resubmitTask();
/**
* run task
*/
protected abstract boolean runTask();
/**
* dispatch task
*/
protected abstract boolean dispatchTask();
@Override
public boolean action(TaskAction taskAction) {
String oldTaskInstanceLogPathMdc = LogUtils.getTaskInstanceLogFullPathMdc();
if (StringUtils.isNotEmpty(threadLoggerInfoName)) {
LogUtils.setTaskInstanceLogFullPathMDC(threadLoggerInfoName);
}
boolean result = false;
try {
switch (taskAction) {
case STOP:
result = stop();
break;
case PAUSE:
result = pause();
break;
case TIMEOUT:
result = timeout();
break;
case SUBMIT:
result = submit();
break;
case RUN:
result = run();
break;
case DISPATCH:
result = dispatch();
break;
case RESUBMIT:
result = resubmit();
break;
default:
log.error("unknown task action: {}", taskAction);
}
return result;
} finally {
LogUtils.removeTaskInstanceLogFullPathMDC();
// reset MDC value, this should be removed.
if (oldTaskInstanceLogPathMdc != null) {
LogUtils.setTaskInstanceLogFullPathMDC(oldTaskInstanceLogPathMdc);
}
}
}
protected boolean resubmit() {
return resubmitTask();
}
protected boolean submit() {
return submitTask();
}
protected boolean run() {
return runTask();
}
protected boolean dispatch() {
return dispatchTask();
}
protected boolean timeout() {
if (timeout) {
return true;
}
timeout = taskTimeout();
return timeout;
}
protected boolean pause() {
if (paused) {
return true;
}
paused = pauseTask();
return paused;
}
protected boolean stop() {
if (killed) {
return true;
}
killed = killTask();
return killed;
}
@Override
public String getType() {
throw new UnsupportedOperationException("This abstract class doesn's has type");
}
@Override
public SPIIdentify getIdentify() {
return SPIIdentify.builder().name(getType()).build();
}
@Override
public TaskInstance taskInstance() {
return this.taskInstance;
}
/**
* set master task running log.
*/
public void setTaskExecutionLogger() {
threadLoggerInfoName = LogUtils.getTaskInstanceLogFullPath(
taskInstance.getFirstSubmitTime(),
processInstance.getProcessDefinitionCode(),
processInstance.getProcessDefinitionVersion(),
taskInstance.getProcessInstanceId(),
taskInstance.getId());
LogUtils.setTaskInstanceLogFullPathMDC(threadLoggerInfoName);
}
/**
* get TaskExecutionContext
*
* @param taskInstance taskInstance
* @return TaskExecutionContext
*/
protected TaskExecutionContext getTaskExecutionContext(TaskInstance taskInstance) {
if (tenantCode == null) {
log.info("Task state changes to {}", TaskExecutionStatus.FAILURE);
taskInstance.setState(TaskExecutionStatus.FAILURE);
taskInstanceDao.upsertTaskInstance(taskInstance);
return null;
}
taskInstance.getProcessInstance().setTenantCode(tenantCode);
public TaskExecutionContext createTaskExecutionContext(TaskInstance taskInstance) throws TaskExecutionContextCreateException {
ProcessInstance workflowInstance = taskInstance.getProcessInstance();
taskInstance.setResources(getResourceFullNames(taskInstance));
TaskChannel taskChannel = taskPluginManager.getTaskChannel(taskInstance.getTaskType());
ResourceParametersHelper resources = taskChannel.getResources(taskInstance.getTaskParams());
this.setTaskResourceInfo(resources);
ResourceParametersHelper resources =
Optional.ofNullable(taskPluginManager.getTaskChannel(taskInstance.getTaskType()))
.map(taskChannel -> taskChannel.getResources(taskInstance.getTaskParams()))
.orElse(null);
setTaskResourceInfo(resources);
// TODO to be optimized
DataQualityTaskExecutionContext dataQualityTaskExecutionContext = null;
if (TASK_TYPE_DATA_QUALITY.equalsIgnoreCase(taskInstance.getTaskType())) {
dataQualityTaskExecutionContext = new DataQualityTaskExecutionContext();
setDataQualityTaskRelation(dataQualityTaskExecutionContext, taskInstance, tenantCode);
setDataQualityTaskRelation(dataQualityTaskExecutionContext, taskInstance, workflowInstance.getTenantCode());
}
K8sTaskExecutionContext k8sTaskExecutionContext = setK8sTaskRelation(taskInstance);
Map<String, Property> businessParamsMap = curingParamsService.preBuildBusinessParams(processInstance);
Map<String, Property> businessParamsMap = curingParamsService.preBuildBusinessParams(workflowInstance);
AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder()
.taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
Map<String, Property> propertyMap =
curingParamsService.paramParsingPreparation(taskInstance, baseParam, processInstance);
curingParamsService.paramParsingPreparation(taskInstance, baseParam, workflowInstance);
return TaskExecutionContextBuilder.get()
.buildWorkflowInstanceHost(masterConfig.getMasterAddress())
.buildTaskInstanceRelatedInfo(taskInstance)
@ -349,17 +150,33 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
.create();
}
public void setTaskResourceInfo(ResourceParametersHelper resourceParametersHelper) {
private Map<String, String> getResourceFullNames(TaskInstance taskInstance) {
Map<String, String> resourcesMap = new HashMap<>();
AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder()
.taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
if (baseParam != null) {
List<ResourceInfo> projectResourceFiles = baseParam.getResourceFilesList();
if (CollectionUtils.isNotEmpty(projectResourceFiles)) {
// TODO: Modify this part to accomodate(migrate) oldversionresources in the future.
projectResourceFiles.forEach(file -> resourcesMap.put(file.getResourceName(),
processService.queryTenantCodeByResName(file.getResourceName(), ResourceType.FILE)));
}
}
return resourcesMap;
}
private void setTaskResourceInfo(ResourceParametersHelper resourceParametersHelper) {
if (Objects.isNull(resourceParametersHelper)) {
return;
}
resourceParametersHelper.getResourceMap().forEach((type, map) -> {
switch (type) {
case DATASOURCE:
this.setTaskDataSourceResourceInfo(map);
setTaskDataSourceResourceInfo(map);
break;
case UDF:
this.setTaskUdfFuncResourceInfo(map);
setTaskUdfFuncResourceInfo(map);
break;
default:
break;
@ -382,7 +199,6 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
dataSourceParameters.setConnectionParams(datasource.getConnectionParams());
map.put(code, dataSourceParameters);
});
}
private void setTaskUdfFuncResourceInfo(Map<Integer, AbstractResourceParameters> map) {
@ -401,12 +217,6 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
});
}
/**
* set data quality task relation
*
* @param dataQualityTaskExecutionContext dataQualityTaskExecutionContext
* @param taskInstance taskInstance
*/
private void setDataQualityTaskRelation(DataQualityTaskExecutionContext dataQualityTaskExecutionContext,
TaskInstance taskInstance, String tenantCode) {
DataQualityParameters dataQualityParameters =
@ -451,10 +261,64 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
setStatisticsValueWriterConfig(dataQualityTaskExecutionContext);
}
private K8sTaskExecutionContext setK8sTaskRelation(TaskInstance taskInstance) {
K8sTaskExecutionContext k8sTaskExecutionContext = null;
String namespace = "";
switch (taskInstance.getTaskType()) {
case "K8S":
case "KUBEFLOW":
K8sTaskParameters k8sTaskParameters =
JSONUtils.parseObject(taskInstance.getTaskParams(), K8sTaskParameters.class);
namespace = k8sTaskParameters.getNamespace();
break;
case "SPARK":
SparkParameters sparkParameters =
JSONUtils.parseObject(taskInstance.getTaskParams(), SparkParameters.class);
if (StringUtils.isNotEmpty(sparkParameters.getNamespace())) {
namespace = sparkParameters.getNamespace();
}
break;
default:
break;
}
if (StringUtils.isNotEmpty(namespace)) {
String clusterName = JSONUtils.toMap(namespace).get(CLUSTER);
String configYaml = processService.findConfigYamlByName(clusterName);
if (configYaml != null) {
k8sTaskExecutionContext =
new K8sTaskExecutionContext(configYaml, JSONUtils.toMap(namespace).get(NAMESPACE_NAME));
}
}
return k8sTaskExecutionContext;
}
/**
* The SourceConfig will be used in DataQualityApplication that
* get the data which be used to get the statistics value
*
* @param dataQualityTaskExecutionContext
* @param config
*/
private void setSourceConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext,
Map<String, String> config) {
if (StringUtils.isNotEmpty(config.get(SRC_DATASOURCE_ID))) {
DataSource dataSource = processService.findDataSourceById(Integer.parseInt(config.get(SRC_DATASOURCE_ID)));
if (dataSource != null) {
ConnectorType srcConnectorType = ConnectorType.of(
DbType.of(Integer.parseInt(config.get(SRC_CONNECTOR_TYPE))).isHive() ? 1 : 0);
dataQualityTaskExecutionContext.setSourceConnectorType(srcConnectorType.getDescription());
dataQualityTaskExecutionContext.setSourceType(dataSource.getType().getCode());
dataQualityTaskExecutionContext.setSourceConnectionParams(dataSource.getConnectionParams());
}
}
}
/**
* It is used to get comparison params, the param contains
* comparison namecomparison table and execute sql.
* When the type is fixed_value, params will be null.
*
* @param dataQualityTaskExecutionContext
* @param config
* @param ruleInputEntryList
@ -501,15 +365,52 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
}
}
/**
* The TargetConfig will be used in DataQualityApplication that
* get the data which be used to compare to src value
*
* @param dataQualityTaskExecutionContext
* @param config
*/
private void setTargetConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext,
Map<String, String> config) {
if (StringUtils.isNotEmpty(config.get(TARGET_DATASOURCE_ID))) {
DataSource dataSource =
processService.findDataSourceById(Integer.parseInt(config.get(TARGET_DATASOURCE_ID)));
if (dataSource != null) {
ConnectorType targetConnectorType = ConnectorType.of(
DbType.of(Integer.parseInt(config.get(TARGET_CONNECTOR_TYPE))).isHive() ? 1 : 0);
dataQualityTaskExecutionContext.setTargetConnectorType(targetConnectorType.getDescription());
dataQualityTaskExecutionContext.setTargetType(dataSource.getType().getCode());
dataQualityTaskExecutionContext.setTargetConnectionParams(dataSource.getConnectionParams());
}
}
}
/**
* The WriterConfig will be used in DataQualityApplication that
* writes the data quality check result into dolphin scheduler datasource
*
* @param dataQualityTaskExecutionContext
*/
private void setWriterConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext) {
DataSource dataSource = getDefaultDataSource();
ConnectorType writerConnectorType = ConnectorType.of(dataSource.getType().isHive() ? 1 : 0);
dataQualityTaskExecutionContext.setWriterConnectorType(writerConnectorType.getDescription());
dataQualityTaskExecutionContext.setWriterType(dataSource.getType().getCode());
dataQualityTaskExecutionContext.setWriterConnectionParams(dataSource.getConnectionParams());
dataQualityTaskExecutionContext.setWriterTable("t_ds_dq_execute_result");
}
/**
* The default datasource is used to get the dolphinscheduler datasource info,
* and the info will be used in StatisticsValueConfig and WriterConfig
*
* @return DataSource
*/
public DataSource getDefaultDataSource() {
DataSource dataSource = new DataSource();
HikariDataSource hikariDataSource = (HikariDataSource) defaultDataSource;
dataSource.setUserName(hikariDataSource.getUsername());
JdbcInfo jdbcInfo = JdbcUrlParser.getJdbcInfo(hikariDataSource.getJdbcUrl());
if (jdbcInfo != null) {
@ -541,118 +442,4 @@ public abstract class BaseTaskProcessor implements ITaskProcessor {
dataQualityTaskExecutionContext.setStatisticsValueTable("t_ds_dq_task_statistics_value");
}
/**
* The WriterConfig will be used in DataQualityApplication that
* writes the data quality check result into dolphin scheduler datasource
* @param dataQualityTaskExecutionContext
*/
private void setWriterConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext) {
DataSource dataSource = getDefaultDataSource();
ConnectorType writerConnectorType = ConnectorType.of(dataSource.getType().isHive() ? 1 : 0);
dataQualityTaskExecutionContext.setWriterConnectorType(writerConnectorType.getDescription());
dataQualityTaskExecutionContext.setWriterType(dataSource.getType().getCode());
dataQualityTaskExecutionContext.setWriterConnectionParams(dataSource.getConnectionParams());
dataQualityTaskExecutionContext.setWriterTable("t_ds_dq_execute_result");
}
/**
* The TargetConfig will be used in DataQualityApplication that
* get the data which be used to compare to src value
* @param dataQualityTaskExecutionContext
* @param config
*/
private void setTargetConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext,
Map<String, String> config) {
if (StringUtils.isNotEmpty(config.get(TARGET_DATASOURCE_ID))) {
DataSource dataSource =
processService.findDataSourceById(Integer.parseInt(config.get(TARGET_DATASOURCE_ID)));
if (dataSource != null) {
ConnectorType targetConnectorType = ConnectorType.of(
DbType.of(Integer.parseInt(config.get(TARGET_CONNECTOR_TYPE))).isHive() ? 1 : 0);
dataQualityTaskExecutionContext.setTargetConnectorType(targetConnectorType.getDescription());
dataQualityTaskExecutionContext.setTargetType(dataSource.getType().getCode());
dataQualityTaskExecutionContext.setTargetConnectionParams(dataSource.getConnectionParams());
}
}
}
/**
* The SourceConfig will be used in DataQualityApplication that
* get the data which be used to get the statistics value
* @param dataQualityTaskExecutionContext
* @param config
*/
private void setSourceConfig(DataQualityTaskExecutionContext dataQualityTaskExecutionContext,
Map<String, String> config) {
if (StringUtils.isNotEmpty(config.get(SRC_DATASOURCE_ID))) {
DataSource dataSource = processService.findDataSourceById(Integer.parseInt(config.get(SRC_DATASOURCE_ID)));
if (dataSource != null) {
ConnectorType srcConnectorType = ConnectorType.of(
DbType.of(Integer.parseInt(config.get(SRC_CONNECTOR_TYPE))).isHive() ? 1 : 0);
dataQualityTaskExecutionContext.setSourceConnectorType(srcConnectorType.getDescription());
dataQualityTaskExecutionContext.setSourceType(dataSource.getType().getCode());
dataQualityTaskExecutionContext.setSourceConnectionParams(dataSource.getConnectionParams());
}
}
}
/**
* get resource map key is full name and value is tenantCode
*/
public Map<String, String> getResourceFullNames(TaskInstance taskInstance) {
Map<String, String> resourcesMap = new HashMap<>();
AbstractParameters baseParam = taskPluginManager.getParameters(ParametersNode.builder()
.taskType(taskInstance.getTaskType()).taskParams(taskInstance.getTaskParams()).build());
if (baseParam != null) {
List<ResourceInfo> projectResourceFiles = baseParam.getResourceFilesList();
if (CollectionUtils.isNotEmpty(projectResourceFiles)) {
projectResourceFiles.forEach(file -> resourcesMap.put(file.getResourceName(),
storageOperate.getResourceFileName(file.getResourceName())));
}
}
return resourcesMap;
}
/**
* get k8s task execution context based on task type and deploy mode
*
* @param taskInstance taskInstance
*/
private K8sTaskExecutionContext setK8sTaskRelation(TaskInstance taskInstance) {
K8sTaskExecutionContext k8sTaskExecutionContext = null;
String namespace = "";
switch (taskInstance.getTaskType()) {
case "K8S":
case "KUBEFLOW":
K8sTaskParameters k8sTaskParameters =
JSONUtils.parseObject(taskInstance.getTaskParams(), K8sTaskParameters.class);
namespace = k8sTaskParameters.getNamespace();
break;
case "SPARK":
SparkParameters sparkParameters =
JSONUtils.parseObject(taskInstance.getTaskParams(), SparkParameters.class);
if (StringUtils.isNotEmpty(sparkParameters.getNamespace())) {
namespace = sparkParameters.getNamespace();
}
break;
default:
break;
}
if (StringUtils.isNotEmpty(namespace)) {
String clusterName = JSONUtils.toMap(namespace).get(CLUSTER);
String configYaml = processService.findConfigYamlByName(clusterName);
if (configYaml != null) {
k8sTaskExecutionContext =
new K8sTaskExecutionContext(configYaml, JSONUtils.toMap(namespace).get(NAMESPACE_NAME));
}
}
return k8sTaskExecutionContext;
}
private String getTenantCode() {
int userId = taskInstance.getProcessDefine() == null ? 0 : taskInstance.getProcessDefine().getUserId();
return processService.getTenantForProcess(taskInstance.getProcessInstance().getTenantCode(), userId);
}
}

50
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterMessageSender.java

@ -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.server.master.runner.message;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.remote.command.BaseMessage;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.exceptions.RemotingException;
// todo: unify with WorkerMessageSender
public interface MasterMessageSender<T extends BaseMessage> {
/**
* Send the message
*
* @throws RemotingException Cannot connect to the target host.
*/
void sendMessage(T message) throws RemotingException;
default void sendMessage(TaskExecutionContext taskExecutionContext) throws RemotingException {
T message = buildMessage(taskExecutionContext);
sendMessage(message);
}
/**
* Build the message from task context and message received address.
*/
T buildMessage(TaskExecutionContext taskExecutionContext);
/**
* The message type can be sent by this sender.
*/
MessageType getMessageType();
}

50
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterMessageSenderManager.java

@ -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.server.master.runner.message;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class MasterMessageSenderManager {
@Autowired
private MasterTaskExecuteResultMessageSender masterTaskExecuteResultMessageSender;
@Autowired
private MasterTaskExecuteRunningMessageSender masterTaskExecuteRunningMessageSender;
@Autowired
private MasterTaskUpdateRuntimeMessageSender masterTaskUpdateRuntimeMessageSender;
public MasterTaskExecuteResultMessageSender getMasterTaskExecuteResultMessageSender() {
return masterTaskExecuteResultMessageSender;
}
public MasterTaskExecuteRunningMessageSender getMasterTaskExecuteRunningMessageSender() {
return masterTaskExecuteRunningMessageSender;
}
public MasterTaskUpdateRuntimeMessageSender getMasterTaskUpdatePidMessageSender() {
return masterTaskUpdateRuntimeMessageSender;
}
}

70
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskExecuteResultMessageSender.java

@ -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.server.master.runner.message;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskExecuteResultMessage;
import org.apache.dolphinscheduler.remote.exceptions.RemotingException;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Component
public class MasterTaskExecuteResultMessageSender implements MasterMessageSender<TaskExecuteResultMessage> {
@Autowired
private MasterConfig masterConfig;
@Autowired
private MasterRpcClient masterRpcClient;
@Override
public void sendMessage(TaskExecuteResultMessage message) throws RemotingException {
masterRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command());
}
@Override
public TaskExecuteResultMessage buildMessage(TaskExecutionContext taskExecutionContext) {
TaskExecuteResultMessage taskExecuteResultMessage =
new TaskExecuteResultMessage(masterConfig.getMasterAddress(),
taskExecutionContext.getWorkflowInstanceHost(),
System.currentTimeMillis());
taskExecuteResultMessage.setProcessInstanceId(taskExecutionContext.getProcessInstanceId());
taskExecuteResultMessage.setTaskInstanceId(taskExecutionContext.getTaskInstanceId());
taskExecuteResultMessage.setStatus(taskExecutionContext.getCurrentExecutionStatus().getCode());
taskExecuteResultMessage.setLogPath(taskExecutionContext.getLogPath());
taskExecuteResultMessage.setExecutePath(taskExecutionContext.getExecutePath());
taskExecuteResultMessage.setAppIds(taskExecutionContext.getAppIds());
taskExecuteResultMessage.setProcessId(taskExecutionContext.getProcessId());
taskExecuteResultMessage.setHost(taskExecutionContext.getHost());
taskExecuteResultMessage.setStartTime(taskExecutionContext.getStartTime());
taskExecuteResultMessage.setEndTime(taskExecutionContext.getEndTime());
taskExecuteResultMessage.setVarPool(taskExecutionContext.getVarPool());
taskExecuteResultMessage.setExecutePath(taskExecutionContext.getExecutePath());
return taskExecuteResultMessage;
}
@Override
public MessageType getMessageType() {
return MessageType.TASK_EXECUTE_RESULT_MESSAGE;
}
}

68
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskExecuteRunningMessageSender.java

@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.message;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskExecuteRunningMessage;
import org.apache.dolphinscheduler.remote.exceptions.RemotingException;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient;
import lombok.NonNull;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Component
public class MasterTaskExecuteRunningMessageSender implements MasterMessageSender<TaskExecuteRunningMessage> {
@Autowired
private MasterRpcClient masterRpcClient;
@Autowired
private MasterConfig masterConfig;
@Override
public void sendMessage(TaskExecuteRunningMessage message) throws RemotingException {
masterRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command());
}
@Override
public TaskExecuteRunningMessage buildMessage(@NonNull TaskExecutionContext taskExecutionContext) {
TaskExecuteRunningMessage taskExecuteRunningMessage =
new TaskExecuteRunningMessage(masterConfig.getMasterAddress(),
taskExecutionContext.getWorkflowInstanceHost(),
System.currentTimeMillis());
taskExecuteRunningMessage.setTaskInstanceId(taskExecutionContext.getTaskInstanceId());
taskExecuteRunningMessage.setProcessInstanceId(taskExecutionContext.getProcessInstanceId());
taskExecuteRunningMessage.setStatus(taskExecutionContext.getCurrentExecutionStatus());
taskExecuteRunningMessage.setLogPath(taskExecutionContext.getLogPath());
taskExecuteRunningMessage.setHost(taskExecutionContext.getHost());
taskExecuteRunningMessage.setStartTime(taskExecutionContext.getStartTime());
taskExecuteRunningMessage.setExecutePath(taskExecutionContext.getExecutePath());
taskExecuteRunningMessage.setAppIds(taskExecutionContext.getAppIds());
return taskExecuteRunningMessage;
}
@Override
public MessageType getMessageType() {
return MessageType.TASK_EXECUTE_RUNNING_MESSAGE;
}
}

64
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/message/MasterTaskUpdateRuntimeMessageSender.java

@ -0,0 +1,64 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.message;
import org.apache.dolphinscheduler.plugin.task.api.TaskExecutionContext;
import org.apache.dolphinscheduler.remote.command.MessageType;
import org.apache.dolphinscheduler.remote.command.task.TaskUpdateRuntimeMessage;
import org.apache.dolphinscheduler.remote.exceptions.RemotingException;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient;
import lombok.NonNull;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Component
public class MasterTaskUpdateRuntimeMessageSender implements MasterMessageSender<TaskUpdateRuntimeMessage> {
@Autowired
private MasterRpcClient masterRpcClient;
@Autowired
private MasterConfig masterConfig;
@Override
public void sendMessage(TaskUpdateRuntimeMessage message) throws RemotingException {
masterRpcClient.send(Host.of(message.getMessageReceiverAddress()), message.convert2Command());
}
@Override
public TaskUpdateRuntimeMessage buildMessage(@NonNull TaskExecutionContext taskExecutionContext) {
TaskUpdateRuntimeMessage taskUpdatePidRequest =
new TaskUpdateRuntimeMessage(masterConfig.getMasterAddress(),
taskExecutionContext.getWorkflowInstanceHost(),
System.currentTimeMillis());
taskUpdatePidRequest.setTaskInstanceId(taskExecutionContext.getTaskInstanceId());
taskUpdatePidRequest.setProcessInstanceId(taskExecutionContext.getProcessInstanceId());
taskUpdatePidRequest.setHost(taskExecutionContext.getHost());
taskUpdatePidRequest.setStartTime(taskExecutionContext.getStartTime());
return taskUpdatePidRequest;
}
@Override
public MessageType getMessageType() {
return MessageType.TASK_UPDATE_RUNTIME_MESSAGE;
}
}

36
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskDispatchOperator.java

@ -0,0 +1,36 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.operator;
import org.apache.dolphinscheduler.server.master.runner.GlobalTaskDispatchWaitingQueue;
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Component
public class TaskDispatchOperator implements TaskOperator {
@Autowired
private GlobalTaskDispatchWaitingQueue globalTaskDispatchWaitingQueue;
@Override
public void handle(DefaultTaskExecuteRunnable taskExecuteRunnable) {
globalTaskDispatchWaitingQueue.submitNeedToDispatchTaskExecuteRunnable(taskExecuteRunnable);
}
}

79
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskKillOperator.java

@ -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.server.master.runner.operator;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.repository.TaskInstanceDao;
import org.apache.dolphinscheduler.plugin.task.api.enums.TaskExecutionStatus;
import org.apache.dolphinscheduler.remote.command.task.TaskKillRequest;
import org.apache.dolphinscheduler.remote.exceptions.RemotingException;
import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.server.master.rpc.MasterRpcClient;
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable;
import org.apache.commons.lang3.StringUtils;
import java.util.Date;
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
@Slf4j
@Component
public class TaskKillOperator implements TaskOperator {
@Autowired
private TaskInstanceDao taskInstanceDao;
@Autowired
private MasterRpcClient masterRpcClient;
@Override
public void handle(DefaultTaskExecuteRunnable taskExecuteRunnable) {
TaskInstance taskInstance = taskExecuteRunnable.getTaskInstance();
log.info("Begin to kill task instance: {}", taskInstance.getName());
if (taskInstance.getState().isFinished()) {
log.info("The task stance {} is finished, no need to kill", taskInstance.getName());
return;
}
try {
killTaskInstanceInDB(taskInstance);
killRemoteTaskInstanceInThreadPool(taskInstance);
} catch (Exception ex) {
// todo: do we need to throw this exception?
log.error("Kill task instance {} failed", taskInstance.getName(), ex);
}
}
private void killTaskInstanceInDB(TaskInstance taskInstance) {
taskInstance.setState(TaskExecutionStatus.KILL);
taskInstance.setEndTime(new Date());
taskInstanceDao.updateTaskInstance(taskInstance);
}
private void killRemoteTaskInstanceInThreadPool(TaskInstance taskInstance) throws RemotingException {
if (StringUtils.isEmpty(taskInstance.getHost())) {
return;
}
TaskKillRequest killCommand = new TaskKillRequest(taskInstance.getId());
masterRpcClient.send(Host.of(taskInstance.getHost()), killCommand.convert2Command());
}
}

26
dolphinscheduler-master/src/main/java/org/apache/dolphinscheduler/server/master/runner/operator/TaskOperator.java

@ -0,0 +1,26 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner.operator;
import org.apache.dolphinscheduler.server.master.runner.execute.DefaultTaskExecuteRunnable;
public interface TaskOperator {
void handle(DefaultTaskExecuteRunnable taskExecuteRunnable);
}

Some files were not shown because too many files have changed in this diff Show More

Loading…
Cancel
Save