Browse Source

Merge pull request #4097 from CalvinKirs/ack_status

[FIX-#4084][server]fix taskInstance state change error
pull/3/MERGE
dailidong 4 years ago committed by GitHub
parent
commit
656ec295b9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 178
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java
  2. 90
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseService.java
  3. 99
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseServiceTest.java
  4. 403
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java
  5. 1
      pom.xml

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

@ -19,8 +19,8 @@ package org.apache.dolphinscheduler.server.master.consumer;
import org.apache.dolphinscheduler.common.Constants; import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.SqoopJobType;
import org.apache.dolphinscheduler.common.enums.ResourceType; import org.apache.dolphinscheduler.common.enums.ResourceType;
import org.apache.dolphinscheduler.common.enums.SqoopJobType;
import org.apache.dolphinscheduler.common.enums.TaskType; import org.apache.dolphinscheduler.common.enums.TaskType;
import org.apache.dolphinscheduler.common.enums.UdfType; import org.apache.dolphinscheduler.common.enums.UdfType;
import org.apache.dolphinscheduler.common.model.TaskNode; import org.apache.dolphinscheduler.common.model.TaskNode;
@ -33,10 +33,24 @@ import org.apache.dolphinscheduler.common.task.sqoop.SqoopParameters;
import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceMysqlParameter; import org.apache.dolphinscheduler.common.task.sqoop.sources.SourceMysqlParameter;
import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetMysqlParameter; import org.apache.dolphinscheduler.common.task.sqoop.targets.TargetMysqlParameter;
import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.common.utils.*; import org.apache.dolphinscheduler.common.utils.CollectionUtils;
import org.apache.dolphinscheduler.dao.entity.*; import org.apache.dolphinscheduler.common.utils.EnumUtils;
import org.apache.dolphinscheduler.common.utils.FileUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import org.apache.dolphinscheduler.common.utils.TaskParametersUtils;
import org.apache.dolphinscheduler.dao.entity.DataSource;
import org.apache.dolphinscheduler.dao.entity.Resource;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.entity.Tenant;
import org.apache.dolphinscheduler.dao.entity.UdfFunc;
import org.apache.dolphinscheduler.server.builder.TaskExecutionContextBuilder; import org.apache.dolphinscheduler.server.builder.TaskExecutionContextBuilder;
import org.apache.dolphinscheduler.server.entity.*; import org.apache.dolphinscheduler.server.entity.DataxTaskExecutionContext;
import org.apache.dolphinscheduler.server.entity.ProcedureTaskExecutionContext;
import org.apache.dolphinscheduler.server.entity.SQLTaskExecutionContext;
import org.apache.dolphinscheduler.server.entity.SqoopTaskExecutionContext;
import org.apache.dolphinscheduler.server.entity.TaskExecutionContext;
import org.apache.dolphinscheduler.server.entity.TaskPriority;
import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher; import org.apache.dolphinscheduler.server.master.dispatch.ExecutorDispatcher;
import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext; import org.apache.dolphinscheduler.server.master.dispatch.context.ExecutionContext;
@ -44,21 +58,27 @@ 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.ExecuteException;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.queue.TaskPriorityQueue; import org.apache.dolphinscheduler.service.queue.TaskPriorityQueue;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import javax.annotation.PostConstruct;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired; import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component; import org.springframework.stereotype.Component;
import javax.annotation.PostConstruct;
import java.util.*;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/** /**
* TaskUpdateQueue consumer * TaskUpdateQueue consumer
*/ */
@Component @Component
public class TaskPriorityQueueConsumer extends Thread{ public class TaskPriorityQueueConsumer extends Thread {
/** /**
* logger of TaskUpdateQueueConsumer * logger of TaskUpdateQueueConsumer
@ -91,7 +111,7 @@ public class TaskPriorityQueueConsumer extends Thread{
private MasterConfig masterConfig; private MasterConfig masterConfig;
@PostConstruct @PostConstruct
public void init(){ public void init() {
super.setName("TaskUpdateQueueConsumerThread"); super.setName("TaskUpdateQueueConsumerThread");
super.start(); super.start();
} }
@ -99,12 +119,12 @@ public class TaskPriorityQueueConsumer extends Thread{
@Override @Override
public void run() { public void run() {
List<String> failedDispatchTasks = new ArrayList<>(); List<String> failedDispatchTasks = new ArrayList<>();
while (Stopper.isRunning()){ while (Stopper.isRunning()) {
try { try {
int fetchTaskNum = masterConfig.getMasterDispatchTaskNumber(); int fetchTaskNum = masterConfig.getMasterDispatchTaskNumber();
failedDispatchTasks.clear(); failedDispatchTasks.clear();
for(int i = 0; i < fetchTaskNum; i++){ for (int i = 0; i < fetchTaskNum; i++) {
if(taskPriorityQueue.size() <= 0){ if (taskPriorityQueue.size() <= 0) {
Thread.sleep(Constants.SLEEP_TIME_MILLIS); Thread.sleep(Constants.SLEEP_TIME_MILLIS);
continue; continue;
} }
@ -112,62 +132,62 @@ public class TaskPriorityQueueConsumer extends Thread{
String taskPriorityInfo = taskPriorityQueue.take(); String taskPriorityInfo = taskPriorityQueue.take();
TaskPriority taskPriority = TaskPriority.of(taskPriorityInfo); TaskPriority taskPriority = TaskPriority.of(taskPriorityInfo);
boolean dispatchResult = dispatch(taskPriority.getTaskId()); boolean dispatchResult = dispatch(taskPriority.getTaskId());
if(!dispatchResult){ if (!dispatchResult) {
failedDispatchTasks.add(taskPriorityInfo); failedDispatchTasks.add(taskPriorityInfo);
} }
} }
for(String dispatchFailedTask : failedDispatchTasks){ for (String dispatchFailedTask : failedDispatchTasks) {
taskPriorityQueue.put(dispatchFailedTask); taskPriorityQueue.put(dispatchFailedTask);
} }
}catch (Exception e){ } catch (Exception e) {
logger.error("dispatcher task error",e); logger.error("dispatcher task error", e);
} }
} }
} }
/** /**
* dispatch task * dispatch task
* *
* @param taskInstanceId taskInstanceId * @param taskInstanceId taskInstanceId
* @return result * @return result
*/ */
private boolean dispatch(int taskInstanceId){ private boolean dispatch(int taskInstanceId) {
boolean result = false; boolean result = false;
try { try {
TaskExecutionContext context = getTaskExecutionContext(taskInstanceId); TaskExecutionContext context = getTaskExecutionContext(taskInstanceId);
ExecutionContext executionContext = new ExecutionContext(context.toCommand(), ExecutorType.WORKER, context.getWorkerGroup()); ExecutionContext executionContext = new ExecutionContext(context.toCommand(), ExecutorType.WORKER, context.getWorkerGroup());
if (taskInstanceIsFinalState(taskInstanceId)){ if (taskInstanceIsFinalState(taskInstanceId)) {
// when task finish, ignore this task, there is no need to dispatch anymore // when task finish, ignore this task, there is no need to dispatch anymore
return true; return true;
}else{ } else {
result = dispatcher.dispatch(executionContext); result = dispatcher.dispatch(executionContext);
} }
} catch (ExecuteException e) { } catch (ExecuteException e) {
logger.error("dispatch error",e); logger.error("dispatch error", e);
} }
return result; return result;
} }
/** /**
* taskInstance is final state * taskInstance is final state
* successfailurekillstoppausethreadwaiting is final state * successfailurekillstoppausethreadwaiting is final state
*
* @param taskInstanceId taskInstanceId * @param taskInstanceId taskInstanceId
* @return taskInstance is final state * @return taskInstance is final state
*/ */
public Boolean taskInstanceIsFinalState(int taskInstanceId){ public Boolean taskInstanceIsFinalState(int taskInstanceId) {
TaskInstance taskInstance = processService.findTaskInstanceById(taskInstanceId); TaskInstance taskInstance = processService.findTaskInstanceById(taskInstanceId);
return taskInstance.getState().typeIsFinished(); return taskInstance.getState().typeIsFinished();
} }
/** /**
* get TaskExecutionContext * get TaskExecutionContext
*
* @param taskInstanceId taskInstanceId * @param taskInstanceId taskInstanceId
* @return TaskExecutionContext * @return TaskExecutionContext
*/ */
protected TaskExecutionContext getTaskExecutionContext(int taskInstanceId){ protected TaskExecutionContext getTaskExecutionContext(int taskInstanceId) {
TaskInstance taskInstance = processService.getTaskInstanceDetailByTaskId(taskInstanceId); TaskInstance taskInstance = processService.getTaskInstanceDetailByTaskId(taskInstanceId);
// task type // task type
@ -181,12 +201,12 @@ public class TaskPriorityQueueConsumer extends Thread{
// verify tenant is null // verify tenant is null
if (verifyTenantIsNull(tenant, taskInstance)) { if (verifyTenantIsNull(tenant, taskInstance)) {
processService.changeTaskState(ExecutionStatus.FAILURE, processService.changeTaskState(taskInstance, ExecutionStatus.FAILURE,
taskInstance.getStartTime(), taskInstance.getStartTime(),
taskInstance.getHost(), taskInstance.getHost(),
null, null,
null, null,
taskInstance.getId()); taskInstance.getId());
return null; return null;
} }
// set queue for process instance, user-specified queue takes precedence over tenant queue // set queue for process instance, user-specified queue takes precedence over tenant queue
@ -196,50 +216,47 @@ public class TaskPriorityQueueConsumer extends Thread{
taskInstance.setExecutePath(getExecLocalPath(taskInstance)); taskInstance.setExecutePath(getExecLocalPath(taskInstance));
taskInstance.setResources(getResourceFullNames(taskNode)); taskInstance.setResources(getResourceFullNames(taskNode));
SQLTaskExecutionContext sqlTaskExecutionContext = new SQLTaskExecutionContext(); SQLTaskExecutionContext sqlTaskExecutionContext = new SQLTaskExecutionContext();
DataxTaskExecutionContext dataxTaskExecutionContext = new DataxTaskExecutionContext(); DataxTaskExecutionContext dataxTaskExecutionContext = new DataxTaskExecutionContext();
ProcedureTaskExecutionContext procedureTaskExecutionContext = new ProcedureTaskExecutionContext(); ProcedureTaskExecutionContext procedureTaskExecutionContext = new ProcedureTaskExecutionContext();
SqoopTaskExecutionContext sqoopTaskExecutionContext = new SqoopTaskExecutionContext(); SqoopTaskExecutionContext sqoopTaskExecutionContext = new SqoopTaskExecutionContext();
// SQL task // SQL task
if (taskType == TaskType.SQL){ if (taskType == TaskType.SQL) {
setSQLTaskRelation(sqlTaskExecutionContext, taskNode); setSQLTaskRelation(sqlTaskExecutionContext, taskNode);
} }
// DATAX task // DATAX task
if (taskType == TaskType.DATAX){ if (taskType == TaskType.DATAX) {
setDataxTaskRelation(dataxTaskExecutionContext, taskNode); setDataxTaskRelation(dataxTaskExecutionContext, taskNode);
} }
// procedure task // procedure task
if (taskType == TaskType.PROCEDURE){ if (taskType == TaskType.PROCEDURE) {
setProcedureTaskRelation(procedureTaskExecutionContext, taskNode); setProcedureTaskRelation(procedureTaskExecutionContext, taskNode);
} }
if (taskType == TaskType.SQOOP){ if (taskType == TaskType.SQOOP) {
setSqoopTaskRelation(sqoopTaskExecutionContext,taskNode); setSqoopTaskRelation(sqoopTaskExecutionContext, taskNode);
} }
return TaskExecutionContextBuilder.get() return TaskExecutionContextBuilder.get()
.buildTaskInstanceRelatedInfo(taskInstance) .buildTaskInstanceRelatedInfo(taskInstance)
.buildProcessInstanceRelatedInfo(taskInstance.getProcessInstance()) .buildProcessInstanceRelatedInfo(taskInstance.getProcessInstance())
.buildProcessDefinitionRelatedInfo(taskInstance.getProcessDefine()) .buildProcessDefinitionRelatedInfo(taskInstance.getProcessDefine())
.buildSQLTaskRelatedInfo(sqlTaskExecutionContext) .buildSQLTaskRelatedInfo(sqlTaskExecutionContext)
.buildDataxTaskRelatedInfo(dataxTaskExecutionContext) .buildDataxTaskRelatedInfo(dataxTaskExecutionContext)
.buildProcedureTaskRelatedInfo(procedureTaskExecutionContext) .buildProcedureTaskRelatedInfo(procedureTaskExecutionContext)
.buildSqoopTaskRelatedInfo(sqoopTaskExecutionContext) .buildSqoopTaskRelatedInfo(sqoopTaskExecutionContext)
.create(); .create();
} }
/** /**
* set procedure task relation * set procedure task relation
*
* @param procedureTaskExecutionContext procedureTaskExecutionContext * @param procedureTaskExecutionContext procedureTaskExecutionContext
* @param taskNode taskNode * @param taskNode taskNode
*/ */
private void setProcedureTaskRelation(ProcedureTaskExecutionContext procedureTaskExecutionContext, TaskNode taskNode) { private void setProcedureTaskRelation(ProcedureTaskExecutionContext procedureTaskExecutionContext, TaskNode taskNode) {
ProcedureParameters procedureParameters = JSONUtils.parseObject(taskNode.getParams(), ProcedureParameters.class); ProcedureParameters procedureParameters = JSONUtils.parseObject(taskNode.getParams(), ProcedureParameters.class);
@ -250,8 +267,9 @@ public class TaskPriorityQueueConsumer extends Thread{
/** /**
* set datax task relation * set datax task relation
*
* @param dataxTaskExecutionContext dataxTaskExecutionContext * @param dataxTaskExecutionContext dataxTaskExecutionContext
* @param taskNode taskNode * @param taskNode taskNode
*/ */
private void setDataxTaskRelation(DataxTaskExecutionContext dataxTaskExecutionContext, TaskNode taskNode) { private void setDataxTaskRelation(DataxTaskExecutionContext dataxTaskExecutionContext, TaskNode taskNode) {
DataxParameters dataxParameters = JSONUtils.parseObject(taskNode.getParams(), DataxParameters.class); DataxParameters dataxParameters = JSONUtils.parseObject(taskNode.getParams(), DataxParameters.class);
@ -259,25 +277,24 @@ public class TaskPriorityQueueConsumer extends Thread{
DataSource dataSource = processService.findDataSourceById(dataxParameters.getDataSource()); DataSource dataSource = processService.findDataSourceById(dataxParameters.getDataSource());
DataSource dataTarget = processService.findDataSourceById(dataxParameters.getDataTarget()); DataSource dataTarget = processService.findDataSourceById(dataxParameters.getDataTarget());
if (dataSource != null) {
if (dataSource != null){
dataxTaskExecutionContext.setDataSourceId(dataxParameters.getDataSource()); dataxTaskExecutionContext.setDataSourceId(dataxParameters.getDataSource());
dataxTaskExecutionContext.setSourcetype(dataSource.getType().getCode()); dataxTaskExecutionContext.setSourcetype(dataSource.getType().getCode());
dataxTaskExecutionContext.setSourceConnectionParams(dataSource.getConnectionParams()); dataxTaskExecutionContext.setSourceConnectionParams(dataSource.getConnectionParams());
} }
if (dataTarget != null){ if (dataTarget != null) {
dataxTaskExecutionContext.setDataTargetId(dataxParameters.getDataTarget()); dataxTaskExecutionContext.setDataTargetId(dataxParameters.getDataTarget());
dataxTaskExecutionContext.setTargetType(dataTarget.getType().getCode()); dataxTaskExecutionContext.setTargetType(dataTarget.getType().getCode());
dataxTaskExecutionContext.setTargetConnectionParams(dataTarget.getConnectionParams()); dataxTaskExecutionContext.setTargetConnectionParams(dataTarget.getConnectionParams());
} }
} }
/** /**
* set sqoop task relation * set sqoop task relation
*
* @param sqoopTaskExecutionContext sqoopTaskExecutionContext * @param sqoopTaskExecutionContext sqoopTaskExecutionContext
* @param taskNode taskNode * @param taskNode taskNode
*/ */
private void setSqoopTaskRelation(SqoopTaskExecutionContext sqoopTaskExecutionContext, TaskNode taskNode) { private void setSqoopTaskRelation(SqoopTaskExecutionContext sqoopTaskExecutionContext, TaskNode taskNode) {
SqoopParameters sqoopParameters = JSONUtils.parseObject(taskNode.getParams(), SqoopParameters.class); SqoopParameters sqoopParameters = JSONUtils.parseObject(taskNode.getParams(), SqoopParameters.class);
@ -290,13 +307,13 @@ public class TaskPriorityQueueConsumer extends Thread{
DataSource dataSource = processService.findDataSourceById(sourceMysqlParameter.getSrcDatasource()); DataSource dataSource = processService.findDataSourceById(sourceMysqlParameter.getSrcDatasource());
DataSource dataTarget = processService.findDataSourceById(targetMysqlParameter.getTargetDatasource()); DataSource dataTarget = processService.findDataSourceById(targetMysqlParameter.getTargetDatasource());
if (dataSource != null){ if (dataSource != null) {
sqoopTaskExecutionContext.setDataSourceId(dataSource.getId()); sqoopTaskExecutionContext.setDataSourceId(dataSource.getId());
sqoopTaskExecutionContext.setSourcetype(dataSource.getType().getCode()); sqoopTaskExecutionContext.setSourcetype(dataSource.getType().getCode());
sqoopTaskExecutionContext.setSourceConnectionParams(dataSource.getConnectionParams()); sqoopTaskExecutionContext.setSourceConnectionParams(dataSource.getConnectionParams());
} }
if (dataTarget != null){ if (dataTarget != null) {
sqoopTaskExecutionContext.setDataTargetId(dataTarget.getId()); sqoopTaskExecutionContext.setDataTargetId(dataTarget.getId());
sqoopTaskExecutionContext.setTargetType(dataTarget.getType().getCode()); sqoopTaskExecutionContext.setTargetType(dataTarget.getType().getCode());
sqoopTaskExecutionContext.setTargetConnectionParams(dataTarget.getConnectionParams()); sqoopTaskExecutionContext.setTargetConnectionParams(dataTarget.getConnectionParams());
@ -306,8 +323,9 @@ public class TaskPriorityQueueConsumer extends Thread{
/** /**
* set SQL task relation * set SQL task relation
*
* @param sqlTaskExecutionContext sqlTaskExecutionContext * @param sqlTaskExecutionContext sqlTaskExecutionContext
* @param taskNode taskNode * @param taskNode taskNode
*/ */
private void setSQLTaskRelation(SQLTaskExecutionContext sqlTaskExecutionContext, TaskNode taskNode) { private void setSQLTaskRelation(SQLTaskExecutionContext sqlTaskExecutionContext, TaskNode taskNode) {
SqlParameters sqlParameters = JSONUtils.parseObject(taskNode.getParams(), SqlParameters.class); SqlParameters sqlParameters = JSONUtils.parseObject(taskNode.getParams(), SqlParameters.class);
@ -317,20 +335,20 @@ public class TaskPriorityQueueConsumer extends Thread{
// whether udf type // whether udf type
boolean udfTypeFlag = EnumUtils.isValidEnum(UdfType.class, sqlParameters.getType()) boolean udfTypeFlag = EnumUtils.isValidEnum(UdfType.class, sqlParameters.getType())
&& StringUtils.isNotEmpty(sqlParameters.getUdfs()); && StringUtils.isNotEmpty(sqlParameters.getUdfs());
if (udfTypeFlag){ if (udfTypeFlag) {
String[] udfFunIds = sqlParameters.getUdfs().split(","); String[] udfFunIds = sqlParameters.getUdfs().split(",");
int[] udfFunIdsArray = new int[udfFunIds.length]; int[] udfFunIdsArray = new int[udfFunIds.length];
for(int i = 0 ; i < udfFunIds.length;i++){ for (int i = 0; i < udfFunIds.length; i++) {
udfFunIdsArray[i]=Integer.parseInt(udfFunIds[i]); udfFunIdsArray[i] = Integer.parseInt(udfFunIds[i]);
} }
List<UdfFunc> udfFuncList = processService.queryUdfFunListByIds(udfFunIdsArray); List<UdfFunc> udfFuncList = processService.queryUdfFunListByIds(udfFunIdsArray);
Map<UdfFunc,String> udfFuncMap = new HashMap<>(); Map<UdfFunc, String> udfFuncMap = new HashMap<>();
for(UdfFunc udfFunc : udfFuncList) { for (UdfFunc udfFunc : udfFuncList) {
String tenantCode = processService.queryTenantCodeByResName(udfFunc.getResourceName(), ResourceType.UDF); String tenantCode = processService.queryTenantCodeByResName(udfFunc.getResourceName(), ResourceType.UDF);
udfFuncMap.put(udfFunc,tenantCode); udfFuncMap.put(udfFunc, tenantCode);
} }
sqlTaskExecutionContext.setUdfFuncTenantCodeMap(udfFuncMap); sqlTaskExecutionContext.setUdfFuncTenantCodeMap(udfFuncMap);
@ -342,25 +360,25 @@ public class TaskPriorityQueueConsumer extends Thread{
* *
* @return execute local path * @return execute local path
*/ */
private String getExecLocalPath(TaskInstance taskInstance){ private String getExecLocalPath(TaskInstance taskInstance) {
return FileUtils.getProcessExecDir(taskInstance.getProcessDefine().getProjectId(), return FileUtils.getProcessExecDir(taskInstance.getProcessDefine().getProjectId(),
taskInstance.getProcessDefine().getId(), taskInstance.getProcessDefine().getId(),
taskInstance.getProcessInstance().getId(), taskInstance.getProcessInstance().getId(),
taskInstance.getId()); taskInstance.getId());
} }
/** /**
* whehter tenant is null * whehter tenant is null
* @param tenant tenant *
* @param tenant tenant
* @param taskInstance taskInstance * @param taskInstance taskInstance
* @return result * @return result
*/ */
private boolean verifyTenantIsNull(Tenant tenant, TaskInstance taskInstance) { private boolean verifyTenantIsNull(Tenant tenant, TaskInstance taskInstance) {
if(tenant == null){ if (tenant == null) {
logger.error("tenant not exists,process instance id : {},task instance id : {}", logger.error("tenant not exists,process instance id : {},task instance id : {}",
taskInstance.getProcessInstance().getId(), taskInstance.getProcessInstance().getId(),
taskInstance.getId()); taskInstance.getId());
return true; return true;
} }
return false; return false;
@ -369,8 +387,8 @@ public class TaskPriorityQueueConsumer extends Thread{
/** /**
* get resource map key is full name and value is tenantCode * get resource map key is full name and value is tenantCode
*/ */
private Map<String,String> getResourceFullNames(TaskNode taskNode) { private Map<String, String> getResourceFullNames(TaskNode taskNode) {
Map<String,String> resourceMap = new HashMap<>(); Map<String, String> resourceMap = new HashMap<>();
AbstractParameters baseParam = TaskParametersUtils.getParameters(taskNode.getType(), taskNode.getParams()); AbstractParameters baseParam = TaskParametersUtils.getParameters(taskNode.getType(), taskNode.getParams());
if (baseParam != null) { if (baseParam != null) {
@ -382,7 +400,7 @@ public class TaskPriorityQueueConsumer extends Thread{
if (CollectionUtils.isNotEmpty(oldVersionResources)) { if (CollectionUtils.isNotEmpty(oldVersionResources)) {
oldVersionResources.forEach( oldVersionResources.forEach(
(t)->resourceMap.put(t.getRes(), processService.queryTenantCodeByResName(t.getRes(), ResourceType.FILE)) (t) -> resourceMap.put(t.getRes(), processService.queryTenantCodeByResName(t.getRes(), ResourceType.FILE))
); );
} }
@ -395,7 +413,7 @@ public class TaskPriorityQueueConsumer extends Thread{
List<Resource> resources = processService.listResourceByIds(resourceIds); List<Resource> resources = processService.listResourceByIds(resourceIds);
resources.forEach( resources.forEach(
(t)->resourceMap.put(t.getFullName(),processService.queryTenantCodeByResName(t.getFullName(), ResourceType.FILE)) (t) -> resourceMap.put(t.getFullName(), processService.queryTenantCodeByResName(t.getFullName(), ResourceType.FILE))
); );
} }
} }

90
dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseService.java

@ -17,7 +17,6 @@
package org.apache.dolphinscheduler.server.master.processor.queue; package org.apache.dolphinscheduler.server.master.processor.queue;
import io.netty.channel.Channel;
import org.apache.dolphinscheduler.common.enums.Event; import org.apache.dolphinscheduler.common.enums.Event;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.thread.Stopper; import org.apache.dolphinscheduler.common.thread.Stopper;
@ -25,18 +24,22 @@ import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.remote.command.DBTaskAckCommand; import org.apache.dolphinscheduler.remote.command.DBTaskAckCommand;
import org.apache.dolphinscheduler.remote.command.DBTaskResponseCommand; import org.apache.dolphinscheduler.remote.command.DBTaskResponseCommand;
import org.apache.dolphinscheduler.service.process.ProcessService; import org.apache.dolphinscheduler.service.process.ProcessService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import javax.annotation.PostConstruct;
import javax.annotation.PreDestroy;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.BlockingQueue; import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.LinkedBlockingQueue;
import javax.annotation.PostConstruct;
import javax.annotation.PreDestroy;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import io.netty.channel.Channel;
/** /**
* task manager * task manager
*/ */
@ -65,21 +68,20 @@ public class TaskResponseService {
*/ */
private Thread taskResponseWorker; private Thread taskResponseWorker;
@PostConstruct @PostConstruct
public void start(){ public void start() {
this.taskResponseWorker = new TaskResponseWorker(); this.taskResponseWorker = new TaskResponseWorker();
this.taskResponseWorker.setName("TaskResponseWorker"); this.taskResponseWorker.setName("TaskResponseWorker");
this.taskResponseWorker.start(); this.taskResponseWorker.start();
} }
@PreDestroy @PreDestroy
public void stop(){ public void stop() {
this.taskResponseWorker.interrupt(); this.taskResponseWorker.interrupt();
if(!eventQueue.isEmpty()){ if (!eventQueue.isEmpty()) {
List<TaskResponseEvent> remainEvents = new ArrayList<>(eventQueue.size()); List<TaskResponseEvent> remainEvents = new ArrayList<>(eventQueue.size());
eventQueue.drainTo(remainEvents); eventQueue.drainTo(remainEvents);
for(TaskResponseEvent event : remainEvents){ for (TaskResponseEvent event : remainEvents) {
this.persist(event); this.persist(event);
} }
} }
@ -90,16 +92,15 @@ public class TaskResponseService {
* *
* @param taskResponseEvent taskResponseEvent * @param taskResponseEvent taskResponseEvent
*/ */
public void addResponse(TaskResponseEvent taskResponseEvent){ public void addResponse(TaskResponseEvent taskResponseEvent) {
try { try {
eventQueue.put(taskResponseEvent); eventQueue.put(taskResponseEvent);
} catch (InterruptedException e) { } catch (InterruptedException e) {
logger.error("put task : {} error :{}", taskResponseEvent,e); logger.error("put task : {} error :{}", taskResponseEvent, e);
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
} }
} }
/** /**
* task worker thread * task worker thread
*/ */
@ -108,16 +109,16 @@ public class TaskResponseService {
@Override @Override
public void run() { public void run() {
while (Stopper.isRunning()){ while (Stopper.isRunning()) {
try { try {
// if not task , blocking here // if not task , blocking here
TaskResponseEvent taskResponseEvent = eventQueue.take(); TaskResponseEvent taskResponseEvent = eventQueue.take();
persist(taskResponseEvent); persist(taskResponseEvent);
} catch (InterruptedException e){ } catch (InterruptedException e) {
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
break; break;
} catch (Exception e){ } catch (Exception e) {
logger.error("persist task error",e); logger.error("persist task error", e);
} }
} }
logger.info("TaskResponseWorker stopped"); logger.info("TaskResponseWorker stopped");
@ -126,51 +127,52 @@ public class TaskResponseService {
/** /**
* persist taskResponseEvent * persist taskResponseEvent
*
* @param taskResponseEvent taskResponseEvent * @param taskResponseEvent taskResponseEvent
*/ */
private void persist(TaskResponseEvent taskResponseEvent){ private void persist(TaskResponseEvent taskResponseEvent) {
Event event = taskResponseEvent.getEvent(); Event event = taskResponseEvent.getEvent();
Channel channel = taskResponseEvent.getChannel(); Channel channel = taskResponseEvent.getChannel();
switch (event){ switch (event) {
case ACK: case ACK:
try { try {
TaskInstance taskInstance = processService.findTaskInstanceById(taskResponseEvent.getTaskInstanceId()); TaskInstance taskInstance = processService.findTaskInstanceById(taskResponseEvent.getTaskInstanceId());
if (taskInstance != null){ if (taskInstance != null && !taskInstance.getState().typeIsFinished()) {
processService.changeTaskState(taskResponseEvent.getState(), processService.changeTaskState(taskInstance, taskResponseEvent.getState(),
taskResponseEvent.getStartTime(), taskResponseEvent.getStartTime(),
taskResponseEvent.getWorkerAddress(), taskResponseEvent.getWorkerAddress(),
taskResponseEvent.getExecutePath(), taskResponseEvent.getExecutePath(),
taskResponseEvent.getLogPath(), taskResponseEvent.getLogPath(),
taskResponseEvent.getTaskInstanceId()); taskResponseEvent.getTaskInstanceId());
} }
// if taskInstance is null (maybe deleted) . retry will be meaningless . so ack success // if taskInstance is null (maybe deleted) . retry will be meaningless . so ack success
DBTaskAckCommand taskAckCommand = new DBTaskAckCommand(ExecutionStatus.SUCCESS.getCode(),taskResponseEvent.getTaskInstanceId()); DBTaskAckCommand taskAckCommand = new DBTaskAckCommand(ExecutionStatus.SUCCESS.getCode(), taskResponseEvent.getTaskInstanceId());
channel.writeAndFlush(taskAckCommand.convert2Command()); channel.writeAndFlush(taskAckCommand.convert2Command());
}catch (Exception e){ } catch (Exception e) {
logger.error("worker ack master error",e); logger.error("worker ack master error", e);
DBTaskAckCommand taskAckCommand = new DBTaskAckCommand(ExecutionStatus.FAILURE.getCode(),-1); DBTaskAckCommand taskAckCommand = new DBTaskAckCommand(ExecutionStatus.FAILURE.getCode(), -1);
channel.writeAndFlush(taskAckCommand.convert2Command()); channel.writeAndFlush(taskAckCommand.convert2Command());
} }
break; break;
case RESULT: case RESULT:
try { try {
TaskInstance taskInstance = processService.findTaskInstanceById(taskResponseEvent.getTaskInstanceId()); TaskInstance taskInstance = processService.findTaskInstanceById(taskResponseEvent.getTaskInstanceId());
if (taskInstance != null){ if (taskInstance != null) {
processService.changeTaskState(taskResponseEvent.getState(), processService.changeTaskState(taskInstance, taskResponseEvent.getState(),
taskResponseEvent.getEndTime(), taskResponseEvent.getEndTime(),
taskResponseEvent.getProcessId(), taskResponseEvent.getProcessId(),
taskResponseEvent.getAppIds(), taskResponseEvent.getAppIds(),
taskResponseEvent.getTaskInstanceId(), taskResponseEvent.getTaskInstanceId(),
taskResponseEvent.getVarPool() taskResponseEvent.getVarPool()
); );
} }
// if taskInstance is null (maybe deleted) . retry will be meaningless . so response success // if taskInstance is null (maybe deleted) . retry will be meaningless . so response success
DBTaskResponseCommand taskResponseCommand = new DBTaskResponseCommand(ExecutionStatus.SUCCESS.getCode(),taskResponseEvent.getTaskInstanceId()); DBTaskResponseCommand taskResponseCommand = new DBTaskResponseCommand(ExecutionStatus.SUCCESS.getCode(), taskResponseEvent.getTaskInstanceId());
channel.writeAndFlush(taskResponseCommand.convert2Command()); channel.writeAndFlush(taskResponseCommand.convert2Command());
}catch (Exception e){ } catch (Exception e) {
logger.error("worker response master error",e); logger.error("worker response master error", e);
DBTaskResponseCommand taskResponseCommand = new DBTaskResponseCommand(ExecutionStatus.FAILURE.getCode(),-1); DBTaskResponseCommand taskResponseCommand = new DBTaskResponseCommand(ExecutionStatus.FAILURE.getCode(), -1);
channel.writeAndFlush(taskResponseCommand.convert2Command()); channel.writeAndFlush(taskResponseCommand.convert2Command());
} }
break; break;

99
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/processor/queue/TaskResponseServiceTest.java

@ -14,55 +14,80 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.dolphinscheduler.server.master.processor.queue;
package org.apache.dolphinscheduler.server.master.processor.queue;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus; import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.server.registry.DependencyConfig; import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.server.registry.ZookeeperNodeManager; import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.server.registry.ZookeeperRegistryCenter;
import org.apache.dolphinscheduler.server.zk.SpringZKServer; import java.util.Date;
import org.apache.dolphinscheduler.service.zk.CuratorZookeeperClient;
import org.apache.dolphinscheduler.service.zk.ZookeeperCachedOperator; import org.junit.After;
import org.apache.dolphinscheduler.service.zk.ZookeeperConfig; import org.junit.Before;
import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.springframework.beans.factory.annotation.Autowired; import org.mockito.InjectMocks;
import org.springframework.test.context.ContextConfiguration; import org.mockito.Mock;
import org.springframework.test.context.junit4.SpringJUnit4ClassRunner; import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import java.util.Date; import io.netty.channel.Channel;
@RunWith(SpringJUnit4ClassRunner.class) @RunWith(MockitoJUnitRunner.class)
@ContextConfiguration(classes={DependencyConfig.class, SpringZKServer.class, TaskResponseService.class, ZookeeperRegistryCenter.class,
ZookeeperCachedOperator.class, ZookeeperConfig.class, ZookeeperNodeManager.class, TaskResponseService.class,
CuratorZookeeperClient.class})
public class TaskResponseServiceTest { public class TaskResponseServiceTest {
@Autowired @Mock(name = "processService")
private TaskResponseService taskResponseService; private ProcessService processService;
@Test @InjectMocks
public void testAdd(){ TaskResponseService taskRspService;
TaskResponseEvent taskResponseEvent = TaskResponseEvent.newAck(ExecutionStatus.RUNNING_EXECUTION, new Date(),
"", "", "", 1,null); @Mock
taskResponseService.addResponse(taskResponseEvent); private Channel channel;
Assert.assertTrue(taskResponseService.getEventQueue().size() == 1);
try { private TaskResponseEvent ackEvent;
Thread.sleep(10);
} catch (InterruptedException ignore) { private TaskResponseEvent resultEvent;
}
//after sleep, inner worker will take the event private TaskInstance taskInstance;
Assert.assertTrue(taskResponseService.getEventQueue().size() == 0);
@Before
public void before() {
taskRspService.start();
ackEvent = TaskResponseEvent.newAck(ExecutionStatus.RUNNING_EXECUTION,
new Date(),
"127.*.*.*",
"path",
"logPath",
22,
channel);
resultEvent = TaskResponseEvent.newResult(ExecutionStatus.SUCCESS,
new Date(),
1,
"ids",
22,
"varPol",
channel);
taskInstance = new TaskInstance();
taskInstance.setId(22);
taskInstance.setState(ExecutionStatus.RUNNING_EXECUTION);
} }
@Test @Test
public void testStop(){ public void testAddResponse() {
TaskResponseEvent taskResponseEvent = TaskResponseEvent.newAck(ExecutionStatus.RUNNING_EXECUTION, new Date(), Mockito.when(processService.findTaskInstanceById(Mockito.any())).thenReturn(taskInstance);
"", "", "", 1,null); Mockito.when(channel.writeAndFlush(Mockito.any())).thenReturn(null);
taskResponseService.addResponse(taskResponseEvent); taskRspService.addResponse(ackEvent);
taskResponseService.stop(); taskRspService.addResponse(resultEvent);
Assert.assertTrue(taskResponseService.getEventQueue().size() == 0); }
@After
public void after() {
taskRspService.stop();
} }
} }

403
dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java

File diff suppressed because it is too large Load Diff

1
pom.xml

@ -835,6 +835,7 @@
<include>**/server/master/SubProcessTaskTest.java</include> <include>**/server/master/SubProcessTaskTest.java</include>
<include>**/server/master/processor/TaskAckProcessorTest.java</include> <include>**/server/master/processor/TaskAckProcessorTest.java</include>
<include>**/server/master/processor/TaskKillResponseProcessorTest.java</include> <include>**/server/master/processor/TaskKillResponseProcessorTest.java</include>
<include>**/server/master/processor/queue/TaskResponseServiceTest.java</include>
<include>**/server/register/ZookeeperNodeManagerTest.java</include> <include>**/server/register/ZookeeperNodeManagerTest.java</include>
<include>**/server/utils/DataxUtilsTest.java</include> <include>**/server/utils/DataxUtilsTest.java</include>
<include>**/server/utils/ExecutionContextTestUtils.java</include> <include>**/server/utils/ExecutionContextTestUtils.java</include>

Loading…
Cancel
Save