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. 128
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/consumer/TaskPriorityQueueConsumer.java
  2. 68
      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. 147
      dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/process/ProcessService.java
  5. 1
      pom.xml

128
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.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.enums.SqoopJobType;
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.UdfType;
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.targets.TargetMysqlParameter;
import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.common.utils.*;
import org.apache.dolphinscheduler.dao.entity.*;
import org.apache.dolphinscheduler.common.utils.CollectionUtils;
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.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.dispatch.ExecutorDispatcher;
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.service.process.ProcessService;
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.LoggerFactory;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Component;
import javax.annotation.PostConstruct;
import java.util.*;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* TaskUpdateQueue consumer
*/
@Component
public class TaskPriorityQueueConsumer extends Thread{
public class TaskPriorityQueueConsumer extends Thread {
/**
* logger of TaskUpdateQueueConsumer
@ -91,7 +111,7 @@ public class TaskPriorityQueueConsumer extends Thread{
private MasterConfig masterConfig;
@PostConstruct
public void init(){
public void init() {
super.setName("TaskUpdateQueueConsumerThread");
super.start();
}
@ -99,12 +119,12 @@ public class TaskPriorityQueueConsumer extends Thread{
@Override
public void run() {
List<String> failedDispatchTasks = new ArrayList<>();
while (Stopper.isRunning()){
while (Stopper.isRunning()) {
try {
int fetchTaskNum = masterConfig.getMasterDispatchTaskNumber();
failedDispatchTasks.clear();
for(int i = 0; i < fetchTaskNum; i++){
if(taskPriorityQueue.size() <= 0){
for (int i = 0; i < fetchTaskNum; i++) {
if (taskPriorityQueue.size() <= 0) {
Thread.sleep(Constants.SLEEP_TIME_MILLIS);
continue;
}
@ -112,62 +132,62 @@ public class TaskPriorityQueueConsumer extends Thread{
String taskPriorityInfo = taskPriorityQueue.take();
TaskPriority taskPriority = TaskPriority.of(taskPriorityInfo);
boolean dispatchResult = dispatch(taskPriority.getTaskId());
if(!dispatchResult){
if (!dispatchResult) {
failedDispatchTasks.add(taskPriorityInfo);
}
}
for(String dispatchFailedTask : failedDispatchTasks){
for (String dispatchFailedTask : failedDispatchTasks) {
taskPriorityQueue.put(dispatchFailedTask);
}
}catch (Exception e){
logger.error("dispatcher task error",e);
} catch (Exception e) {
logger.error("dispatcher task error", e);
}
}
}
/**
* dispatch task
*
* @param taskInstanceId taskInstanceId
* @return result
*/
private boolean dispatch(int taskInstanceId){
private boolean dispatch(int taskInstanceId) {
boolean result = false;
try {
TaskExecutionContext context = getTaskExecutionContext(taskInstanceId);
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
return true;
}else{
} else {
result = dispatcher.dispatch(executionContext);
}
} catch (ExecuteException e) {
logger.error("dispatch error",e);
logger.error("dispatch error", e);
}
return result;
}
/**
* taskInstance is final state
* successfailurekillstoppausethreadwaiting is final state
*
* @param taskInstanceId taskInstanceId
* @return taskInstance is final state
*/
public Boolean taskInstanceIsFinalState(int taskInstanceId){
public Boolean taskInstanceIsFinalState(int taskInstanceId) {
TaskInstance taskInstance = processService.findTaskInstanceById(taskInstanceId);
return taskInstance.getState().typeIsFinished();
}
/**
* get TaskExecutionContext
*
* @param taskInstanceId taskInstanceId
* @return TaskExecutionContext
*/
protected TaskExecutionContext getTaskExecutionContext(int taskInstanceId){
protected TaskExecutionContext getTaskExecutionContext(int taskInstanceId) {
TaskInstance taskInstance = processService.getTaskInstanceDetailByTaskId(taskInstanceId);
// task type
@ -181,7 +201,7 @@ public class TaskPriorityQueueConsumer extends Thread{
// verify tenant is null
if (verifyTenantIsNull(tenant, taskInstance)) {
processService.changeTaskState(ExecutionStatus.FAILURE,
processService.changeTaskState(taskInstance, ExecutionStatus.FAILURE,
taskInstance.getStartTime(),
taskInstance.getHost(),
null,
@ -196,35 +216,31 @@ public class TaskPriorityQueueConsumer extends Thread{
taskInstance.setExecutePath(getExecLocalPath(taskInstance));
taskInstance.setResources(getResourceFullNames(taskNode));
SQLTaskExecutionContext sqlTaskExecutionContext = new SQLTaskExecutionContext();
DataxTaskExecutionContext dataxTaskExecutionContext = new DataxTaskExecutionContext();
ProcedureTaskExecutionContext procedureTaskExecutionContext = new ProcedureTaskExecutionContext();
SqoopTaskExecutionContext sqoopTaskExecutionContext = new SqoopTaskExecutionContext();
// SQL task
if (taskType == TaskType.SQL){
if (taskType == TaskType.SQL) {
setSQLTaskRelation(sqlTaskExecutionContext, taskNode);
}
// DATAX task
if (taskType == TaskType.DATAX){
if (taskType == TaskType.DATAX) {
setDataxTaskRelation(dataxTaskExecutionContext, taskNode);
}
// procedure task
if (taskType == TaskType.PROCEDURE){
if (taskType == TaskType.PROCEDURE) {
setProcedureTaskRelation(procedureTaskExecutionContext, taskNode);
}
if (taskType == TaskType.SQOOP){
setSqoopTaskRelation(sqoopTaskExecutionContext,taskNode);
if (taskType == TaskType.SQOOP) {
setSqoopTaskRelation(sqoopTaskExecutionContext, taskNode);
}
return TaskExecutionContextBuilder.get()
.buildTaskInstanceRelatedInfo(taskInstance)
.buildProcessInstanceRelatedInfo(taskInstance.getProcessInstance())
@ -238,6 +254,7 @@ public class TaskPriorityQueueConsumer extends Thread{
/**
* set procedure task relation
*
* @param procedureTaskExecutionContext procedureTaskExecutionContext
* @param taskNode taskNode
*/
@ -250,6 +267,7 @@ public class TaskPriorityQueueConsumer extends Thread{
/**
* set datax task relation
*
* @param dataxTaskExecutionContext dataxTaskExecutionContext
* @param taskNode taskNode
*/
@ -259,23 +277,22 @@ public class TaskPriorityQueueConsumer extends Thread{
DataSource dataSource = processService.findDataSourceById(dataxParameters.getDataSource());
DataSource dataTarget = processService.findDataSourceById(dataxParameters.getDataTarget());
if (dataSource != null){
if (dataSource != null) {
dataxTaskExecutionContext.setDataSourceId(dataxParameters.getDataSource());
dataxTaskExecutionContext.setSourcetype(dataSource.getType().getCode());
dataxTaskExecutionContext.setSourceConnectionParams(dataSource.getConnectionParams());
}
if (dataTarget != null){
if (dataTarget != null) {
dataxTaskExecutionContext.setDataTargetId(dataxParameters.getDataTarget());
dataxTaskExecutionContext.setTargetType(dataTarget.getType().getCode());
dataxTaskExecutionContext.setTargetConnectionParams(dataTarget.getConnectionParams());
}
}
/**
* set sqoop task relation
*
* @param sqoopTaskExecutionContext sqoopTaskExecutionContext
* @param taskNode taskNode
*/
@ -290,13 +307,13 @@ public class TaskPriorityQueueConsumer extends Thread{
DataSource dataSource = processService.findDataSourceById(sourceMysqlParameter.getSrcDatasource());
DataSource dataTarget = processService.findDataSourceById(targetMysqlParameter.getTargetDatasource());
if (dataSource != null){
if (dataSource != null) {
sqoopTaskExecutionContext.setDataSourceId(dataSource.getId());
sqoopTaskExecutionContext.setSourcetype(dataSource.getType().getCode());
sqoopTaskExecutionContext.setSourceConnectionParams(dataSource.getConnectionParams());
}
if (dataTarget != null){
if (dataTarget != null) {
sqoopTaskExecutionContext.setDataTargetId(dataTarget.getId());
sqoopTaskExecutionContext.setTargetType(dataTarget.getType().getCode());
sqoopTaskExecutionContext.setTargetConnectionParams(dataTarget.getConnectionParams());
@ -306,6 +323,7 @@ public class TaskPriorityQueueConsumer extends Thread{
/**
* set SQL task relation
*
* @param sqlTaskExecutionContext sqlTaskExecutionContext
* @param taskNode taskNode
*/
@ -319,18 +337,18 @@ public class TaskPriorityQueueConsumer extends Thread{
boolean udfTypeFlag = EnumUtils.isValidEnum(UdfType.class, sqlParameters.getType())
&& StringUtils.isNotEmpty(sqlParameters.getUdfs());
if (udfTypeFlag){
if (udfTypeFlag) {
String[] udfFunIds = sqlParameters.getUdfs().split(",");
int[] udfFunIdsArray = new int[udfFunIds.length];
for(int i = 0 ; i < udfFunIds.length;i++){
udfFunIdsArray[i]=Integer.parseInt(udfFunIds[i]);
for (int i = 0; i < udfFunIds.length; i++) {
udfFunIdsArray[i] = Integer.parseInt(udfFunIds[i]);
}
List<UdfFunc> udfFuncList = processService.queryUdfFunListByIds(udfFunIdsArray);
Map<UdfFunc,String> udfFuncMap = new HashMap<>();
for(UdfFunc udfFunc : udfFuncList) {
Map<UdfFunc, String> udfFuncMap = new HashMap<>();
for (UdfFunc udfFunc : udfFuncList) {
String tenantCode = processService.queryTenantCodeByResName(udfFunc.getResourceName(), ResourceType.UDF);
udfFuncMap.put(udfFunc,tenantCode);
udfFuncMap.put(udfFunc, tenantCode);
}
sqlTaskExecutionContext.setUdfFuncTenantCodeMap(udfFuncMap);
@ -342,22 +360,22 @@ public class TaskPriorityQueueConsumer extends Thread{
*
* @return execute local path
*/
private String getExecLocalPath(TaskInstance taskInstance){
private String getExecLocalPath(TaskInstance taskInstance) {
return FileUtils.getProcessExecDir(taskInstance.getProcessDefine().getProjectId(),
taskInstance.getProcessDefine().getId(),
taskInstance.getProcessInstance().getId(),
taskInstance.getId());
}
/**
* whehter tenant is null
*
* @param tenant tenant
* @param taskInstance taskInstance
* @return result
*/
private boolean verifyTenantIsNull(Tenant tenant, TaskInstance taskInstance) {
if(tenant == null){
if (tenant == null) {
logger.error("tenant not exists,process instance id : {},task instance id : {}",
taskInstance.getProcessInstance().getId(),
taskInstance.getId());
@ -369,8 +387,8 @@ public class TaskPriorityQueueConsumer extends Thread{
/**
* get resource map key is full name and value is tenantCode
*/
private Map<String,String> getResourceFullNames(TaskNode taskNode) {
Map<String,String> resourceMap = new HashMap<>();
private Map<String, String> getResourceFullNames(TaskNode taskNode) {
Map<String, String> resourceMap = new HashMap<>();
AbstractParameters baseParam = TaskParametersUtils.getParameters(taskNode.getType(), taskNode.getParams());
if (baseParam != null) {
@ -382,7 +400,7 @@ public class TaskPriorityQueueConsumer extends Thread{
if (CollectionUtils.isNotEmpty(oldVersionResources)) {
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);
resources.forEach(
(t)->resourceMap.put(t.getFullName(),processService.queryTenantCodeByResName(t.getFullName(), ResourceType.FILE))
(t) -> resourceMap.put(t.getFullName(), processService.queryTenantCodeByResName(t.getFullName(), ResourceType.FILE))
);
}
}

68
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;
import io.netty.channel.Channel;
import org.apache.dolphinscheduler.common.enums.Event;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
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.DBTaskResponseCommand;
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.List;
import java.util.concurrent.BlockingQueue;
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
*/
@ -65,21 +68,20 @@ public class TaskResponseService {
*/
private Thread taskResponseWorker;
@PostConstruct
public void start(){
public void start() {
this.taskResponseWorker = new TaskResponseWorker();
this.taskResponseWorker.setName("TaskResponseWorker");
this.taskResponseWorker.start();
}
@PreDestroy
public void stop(){
public void stop() {
this.taskResponseWorker.interrupt();
if(!eventQueue.isEmpty()){
if (!eventQueue.isEmpty()) {
List<TaskResponseEvent> remainEvents = new ArrayList<>(eventQueue.size());
eventQueue.drainTo(remainEvents);
for(TaskResponseEvent event : remainEvents){
for (TaskResponseEvent event : remainEvents) {
this.persist(event);
}
}
@ -90,16 +92,15 @@ public class TaskResponseService {
*
* @param taskResponseEvent taskResponseEvent
*/
public void addResponse(TaskResponseEvent taskResponseEvent){
public void addResponse(TaskResponseEvent taskResponseEvent) {
try {
eventQueue.put(taskResponseEvent);
} catch (InterruptedException e) {
logger.error("put task : {} error :{}", taskResponseEvent,e);
logger.error("put task : {} error :{}", taskResponseEvent, e);
Thread.currentThread().interrupt();
}
}
/**
* task worker thread
*/
@ -108,16 +109,16 @@ public class TaskResponseService {
@Override
public void run() {
while (Stopper.isRunning()){
while (Stopper.isRunning()) {
try {
// if not task , blocking here
TaskResponseEvent taskResponseEvent = eventQueue.take();
persist(taskResponseEvent);
} catch (InterruptedException e){
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
break;
} catch (Exception e){
logger.error("persist task error",e);
} catch (Exception e) {
logger.error("persist task error", e);
}
}
logger.info("TaskResponseWorker stopped");
@ -126,18 +127,19 @@ public class TaskResponseService {
/**
* persist taskResponseEvent
*
* @param taskResponseEvent taskResponseEvent
*/
private void persist(TaskResponseEvent taskResponseEvent){
private void persist(TaskResponseEvent taskResponseEvent) {
Event event = taskResponseEvent.getEvent();
Channel channel = taskResponseEvent.getChannel();
switch (event){
switch (event) {
case ACK:
try {
TaskInstance taskInstance = processService.findTaskInstanceById(taskResponseEvent.getTaskInstanceId());
if (taskInstance != null){
processService.changeTaskState(taskResponseEvent.getState(),
if (taskInstance != null && !taskInstance.getState().typeIsFinished()) {
processService.changeTaskState(taskInstance, taskResponseEvent.getState(),
taskResponseEvent.getStartTime(),
taskResponseEvent.getWorkerAddress(),
taskResponseEvent.getExecutePath(),
@ -145,19 +147,19 @@ public class TaskResponseService {
taskResponseEvent.getTaskInstanceId());
}
// 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());
}catch (Exception e){
logger.error("worker ack master error",e);
DBTaskAckCommand taskAckCommand = new DBTaskAckCommand(ExecutionStatus.FAILURE.getCode(),-1);
} catch (Exception e) {
logger.error("worker ack master error", e);
DBTaskAckCommand taskAckCommand = new DBTaskAckCommand(ExecutionStatus.FAILURE.getCode(), -1);
channel.writeAndFlush(taskAckCommand.convert2Command());
}
break;
case RESULT:
try {
TaskInstance taskInstance = processService.findTaskInstanceById(taskResponseEvent.getTaskInstanceId());
if (taskInstance != null){
processService.changeTaskState(taskResponseEvent.getState(),
if (taskInstance != null) {
processService.changeTaskState(taskInstance, taskResponseEvent.getState(),
taskResponseEvent.getEndTime(),
taskResponseEvent.getProcessId(),
taskResponseEvent.getAppIds(),
@ -166,11 +168,11 @@ public class TaskResponseService {
);
}
// 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());
}catch (Exception e){
logger.error("worker response master error",e);
DBTaskResponseCommand taskResponseCommand = new DBTaskResponseCommand(ExecutionStatus.FAILURE.getCode(),-1);
} catch (Exception e) {
logger.error("worker response master error", e);
DBTaskResponseCommand taskResponseCommand = new DBTaskResponseCommand(ExecutionStatus.FAILURE.getCode(), -1);
channel.writeAndFlush(taskResponseCommand.convert2Command());
}
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
* 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.server.registry.DependencyConfig;
import org.apache.dolphinscheduler.server.registry.ZookeeperNodeManager;
import org.apache.dolphinscheduler.server.registry.ZookeeperRegistryCenter;
import org.apache.dolphinscheduler.server.zk.SpringZKServer;
import org.apache.dolphinscheduler.service.zk.CuratorZookeeperClient;
import org.apache.dolphinscheduler.service.zk.ZookeeperCachedOperator;
import org.apache.dolphinscheduler.service.zk.ZookeeperConfig;
import org.junit.Assert;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.service.process.ProcessService;
import java.util.Date;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.test.context.ContextConfiguration;
import org.springframework.test.context.junit4.SpringJUnit4ClassRunner;
import org.mockito.InjectMocks;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnitRunner;
import java.util.Date;
import io.netty.channel.Channel;
@RunWith(SpringJUnit4ClassRunner.class)
@ContextConfiguration(classes={DependencyConfig.class, SpringZKServer.class, TaskResponseService.class, ZookeeperRegistryCenter.class,
ZookeeperCachedOperator.class, ZookeeperConfig.class, ZookeeperNodeManager.class, TaskResponseService.class,
CuratorZookeeperClient.class})
@RunWith(MockitoJUnitRunner.class)
public class TaskResponseServiceTest {
@Autowired
private TaskResponseService taskResponseService;
@Mock(name = "processService")
private ProcessService processService;
@Test
public void testAdd(){
TaskResponseEvent taskResponseEvent = TaskResponseEvent.newAck(ExecutionStatus.RUNNING_EXECUTION, new Date(),
"", "", "", 1,null);
taskResponseService.addResponse(taskResponseEvent);
Assert.assertTrue(taskResponseService.getEventQueue().size() == 1);
try {
Thread.sleep(10);
} catch (InterruptedException ignore) {
}
//after sleep, inner worker will take the event
Assert.assertTrue(taskResponseService.getEventQueue().size() == 0);
@InjectMocks
TaskResponseService taskRspService;
@Mock
private Channel channel;
private TaskResponseEvent ackEvent;
private TaskResponseEvent resultEvent;
private TaskInstance taskInstance;
@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
public void testStop(){
TaskResponseEvent taskResponseEvent = TaskResponseEvent.newAck(ExecutionStatus.RUNNING_EXECUTION, new Date(),
"", "", "", 1,null);
taskResponseService.addResponse(taskResponseEvent);
taskResponseService.stop();
Assert.assertTrue(taskResponseService.getEventQueue().size() == 0);
public void testAddResponse() {
Mockito.when(processService.findTaskInstanceById(Mockito.any())).thenReturn(taskInstance);
Mockito.when(channel.writeAndFlush(Mockito.any())).thenReturn(null);
taskRspService.addResponse(ackEvent);
taskRspService.addResponse(resultEvent);
}
@After
public void after() {
taskRspService.stop();
}
}

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

@ -79,8 +79,6 @@ import org.apache.dolphinscheduler.remote.utils.Host;
import org.apache.dolphinscheduler.service.log.LogClientService;
import org.apache.dolphinscheduler.service.quartz.cron.CronUtils;
import org.apache.commons.lang.ArrayUtils;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Calendar;
@ -111,7 +109,7 @@ public class ProcessService {
private final Logger logger = LoggerFactory.getLogger(getClass());
private final int[] stateArray = new int[]{ExecutionStatus.SUBMITTED_SUCCESS.ordinal(),
private final int[] stateArray = new int[] {ExecutionStatus.SUBMITTED_SUCCESS.ordinal(),
ExecutionStatus.RUNNING_EXECUTION.ordinal(),
ExecutionStatus.DELAY_EXECUTION.ordinal(),
ExecutionStatus.READY_PAUSE.ordinal(),
@ -158,6 +156,7 @@ public class ProcessService {
/**
* handle Command (construct ProcessInstance from Command) , wrapped in transaction
*
* @param logger logger
* @param host host
* @param validThreadNum validThreadNum
@ -187,6 +186,7 @@ public class ProcessService {
/**
* save error command, and delete original command
*
* @param command command
* @param message message
*/
@ -199,6 +199,7 @@ public class ProcessService {
/**
* set process waiting thread
*
* @param command command
* @param processInstance processInstance
* @return process instance
@ -216,6 +217,7 @@ public class ProcessService {
/**
* check thread num
*
* @param command command
* @param validThreadNum validThreadNum
* @return if thread is enough
@ -227,6 +229,7 @@ public class ProcessService {
/**
* insert one command
*
* @param command command
* @return create result
*/
@ -240,6 +243,7 @@ public class ProcessService {
/**
* find one command from queue list
*
* @return command
*/
public Command findOneCommand() {
@ -248,15 +252,16 @@ public class ProcessService {
/**
* check the input command exists in queue list
*
* @param command command
* @return create command result
*/
public Boolean verifyIsNeedCreateCommand(Command command) {
Boolean isNeedCreate = true;
Map<CommandType,Integer> cmdTypeMap = new HashMap<CommandType,Integer>();
cmdTypeMap.put(CommandType.REPEAT_RUNNING,1);
cmdTypeMap.put(CommandType.RECOVER_SUSPENDED_PROCESS,1);
cmdTypeMap.put(CommandType.START_FAILURE_TASK_PROCESS,1);
Map<CommandType, Integer> cmdTypeMap = new HashMap<CommandType, Integer>();
cmdTypeMap.put(CommandType.REPEAT_RUNNING, 1);
cmdTypeMap.put(CommandType.RECOVER_SUSPENDED_PROCESS, 1);
cmdTypeMap.put(CommandType.START_FAILURE_TASK_PROCESS, 1);
CommandType commandType = command.getCommandType();
if (cmdTypeMap.containsKey(commandType)) {
@ -265,7 +270,7 @@ public class ProcessService {
List<Command> commands = commandMapper.selectList(null);
// for all commands
for (Command tmpCommand:commands) {
for (Command tmpCommand : commands) {
if (cmdTypeMap.containsKey(tmpCommand.getCommandType())) {
ObjectNode tempObj = JSONUtils.parseObject(tmpCommand.getCommandParam());
if (tempObj != null && processInstanceId == tempObj.path(CMDPARAM_RECOVER_PROCESS_ID_STRING).asInt()) {
@ -280,6 +285,7 @@ public class ProcessService {
/**
* find process instance detail by id
*
* @param processId processId
* @return process instance
*/
@ -289,6 +295,7 @@ public class ProcessService {
/**
* get task node list by definitionId
*
* @param defineId
* @return
*/
@ -313,6 +320,7 @@ public class ProcessService {
/**
* find process instance by id
*
* @param processId processId
* @return process instance
*/
@ -322,6 +330,7 @@ public class ProcessService {
/**
* find process define by id.
*
* @param processDefinitionId processDefinitionId
* @return process definition
*/
@ -331,6 +340,7 @@ public class ProcessService {
/**
* delete work process instance by id
*
* @param processInstanceId processInstanceId
* @return delete process instance result
*/
@ -340,6 +350,7 @@ public class ProcessService {
/**
* delete all sub process by parent instance id
*
* @param processInstanceId processInstanceId
* @return delete all sub process instance result
*/
@ -358,9 +369,10 @@ public class ProcessService {
/**
* remove task log file
*
* @param processInstanceId processInstanceId
*/
public void removeTaskLogFile(Integer processInstanceId){
public void removeTaskLogFile(Integer processInstanceId) {
LogClientService logClient = null;
@ -389,7 +401,7 @@ public class ProcessService {
// remove task log from loggerserver
logClient.removeTaskLog(ip, port, taskLogPath);
}
}finally {
} finally {
if (logClient != null) {
logClient.close();
}
@ -398,6 +410,7 @@ public class ProcessService {
/**
* calculate sub process number in the process define.
*
* @param processDefinitionId processDefinitionId
* @return process thread num count
*/
@ -409,6 +422,7 @@ public class ProcessService {
/**
* recursive query sub process definition id by parent id.
*
* @param parentId parentId
* @param ids ids
*/
@ -428,7 +442,7 @@ public class ProcessService {
if (parameterJson.get(CMDPARAM_SUB_PROCESS_DEFINE_ID) != null) {
SubProcessParameters subProcessParam = JSONUtils.parseObject(parameter, SubProcessParameters.class);
ids.add(subProcessParam.getProcessDefinitionId());
recurseFindSubProcessId(subProcessParam.getProcessDefinitionId(),ids);
recurseFindSubProcessId(subProcessParam.getProcessDefinitionId(), ids);
}
}
@ -440,6 +454,7 @@ public class ProcessService {
* sub work process instance need not to create recovery command.
* create recovery waiting thread command and delete origin command at the same time.
* if the recovery command is exists, only update the field update_time
*
* @param originCommand originCommand
* @param processInstance processInstance
*/
@ -491,6 +506,7 @@ public class ProcessService {
/**
* get schedule time from command
*
* @param command command
* @param cmdParam cmdParam map
* @return date
@ -507,6 +523,7 @@ public class ProcessService {
/**
* generate a new work process instance from command.
*
* @param processDefinition processDefinition
* @param command command
* @param cmdParam cmdParam map
@ -564,6 +581,7 @@ public class ProcessService {
* there is tenant id in definition, use the tenant of the definition.
* if there is not tenant id in the definiton or the tenant not exist
* use definition creator's tenant.
*
* @param tenantId tenantId
* @param userId userId
* @return tenant
@ -587,6 +605,7 @@ public class ProcessService {
/**
* check command parameters is valid
*
* @param command command
* @param cmdParam cmdParam map
* @return whether command param is valid
@ -605,6 +624,7 @@ public class ProcessService {
/**
* construct process instance according to one command.
*
* @param command command
* @param host host
* @return process instance
@ -654,7 +674,7 @@ public class ProcessService {
//reset command parameter
if (processInstance.getCommandParam() != null) {
Map<String, String> processCmdParam = JSONUtils.toMap(processInstance.getCommandParam());
for (Map.Entry<String, String> entry: processCmdParam.entrySet()) {
for (Map.Entry<String, String> entry : processCmdParam.entrySet()) {
if (!cmdParam.containsKey(entry.getKey())) {
cmdParam.put(entry.getKey(), entry.getValue());
}
@ -761,6 +781,7 @@ public class ProcessService {
/**
* return complement data if the process start with complement data
*
* @param processInstance processInstance
* @param command command
* @return command type
@ -775,6 +796,7 @@ public class ProcessService {
/**
* initialize complement data parameters
*
* @param processDefinition processDefinition
* @param processInstance processInstance
* @param cmdParam cmdParam
@ -802,6 +824,7 @@ public class ProcessService {
* set sub work process parameters.
* handle sub work process instance, update relation table and command parameters
* set sub work process flag, extends parent work process command parameters
*
* @param subProcessInstance subProcessInstance
* @return process instance
*/
@ -846,6 +869,7 @@ public class ProcessService {
/**
* join parent global params into sub process.
* only the keys doesn't in sub process global would be joined.
*
* @param parentGlobalParams parentGlobalParams
* @param subGlobalParams subGlobalParams
* @return global params join
@ -855,7 +879,7 @@ public class ProcessService {
List<Property> parentPropertyList = JSONUtils.toList(parentGlobalParams, Property.class);
List<Property> subPropertyList = JSONUtils.toList(subGlobalParams, Property.class);
Map<String,String> subMap = subPropertyList.stream().collect(Collectors.toMap(Property::getProp, Property::getValue));
Map<String, String> subMap = subPropertyList.stream().collect(Collectors.toMap(Property::getProp, Property::getValue));
for (Property parent : parentPropertyList) {
if (!subMap.containsKey(parent.getProp())) {
@ -867,6 +891,7 @@ public class ProcessService {
/**
* initialize task instance
*
* @param taskInstance taskInstance
*/
private void initTaskInstance(TaskInstance taskInstance) {
@ -885,11 +910,12 @@ public class ProcessService {
/**
* submit task to db
* submit sub process to command
*
* @param taskInstance taskInstance
* @return task instance
*/
@Transactional(rollbackFor = Exception.class)
public TaskInstance submitTask(TaskInstance taskInstance){
public TaskInstance submitTask(TaskInstance taskInstance) {
ProcessInstance processInstance = this.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
logger.info("start submit task : {}, instance id:{}, state: {}",
taskInstance.getName(), taskInstance.getProcessInstanceId(), processInstance.getState());
@ -914,6 +940,7 @@ public class ProcessService {
* consider o
* repeat running does not generate new sub process instance
* set map {parent instance id, task instance id, 0(child instance id)}
*
* @param parentInstance parentInstance
* @param parentTask parentTask
* @return process instance map
@ -942,6 +969,7 @@ public class ProcessService {
/**
* find previous task work process map.
*
* @param parentProcessInstance parentProcessInstance
* @param parentTask parentTask
* @return process instance map
@ -967,6 +995,7 @@ public class ProcessService {
/**
* create sub work process command
*
* @param parentProcessInstance parentProcessInstance
* @param task task
*/
@ -994,6 +1023,7 @@ public class ProcessService {
/**
* complement data needs transform parent parameter to child.
*
* @param instanceMap
* @param parentProcessInstance
* @return
@ -1015,6 +1045,7 @@ public class ProcessService {
/**
* create sub work process command
*
* @param parentProcessInstance
* @param childInstance
* @param instanceMap
@ -1048,6 +1079,7 @@ public class ProcessService {
/**
* initialize sub work flow state
* child instance state would be initialized when 'recovery from pause/stop/failure'
*
* @param childInstance
*/
private void initSubInstanceState(ProcessInstance childInstance) {
@ -1076,6 +1108,7 @@ public class ProcessService {
/**
* update sub process definition
*
* @param parentProcessInstance parentProcessInstance
* @param childDefinitionId childDefinitionId
*/
@ -1091,6 +1124,7 @@ public class ProcessService {
/**
* submit task to mysql
*
* @param taskInstance taskInstance
* @param processInstance processInstance
* @return task instance
@ -1140,6 +1174,7 @@ public class ProcessService {
/**
* ${processInstancePriority}_${processInstanceId}_${taskInstancePriority}_${taskInstanceId}_${task executed by ip1},${ip2}...
* The tasks with the highest priority are selected by comparing the priorities of the above four levels from high to low.
*
* @param taskInstance taskInstance
* @return task zk queue str
*/
@ -1203,6 +1238,7 @@ public class ProcessService {
/**
* check process instance strategy
*
* @param taskInstance taskInstance
* @return check strategy result
*/
@ -1224,6 +1260,7 @@ public class ProcessService {
/**
* check the task instance existing in queue
*
* @param taskInstance taskInstance
* @return whether taskinstance exists queue
*/
@ -1239,6 +1276,7 @@ public class ProcessService {
/**
* create a new process instance
*
* @param processInstance processInstance
*/
public void createProcessInstance(ProcessInstance processInstance) {
@ -1250,6 +1288,7 @@ public class ProcessService {
/**
* insert or update work process instance to data base
*
* @param processInstance processInstance
*/
public void saveProcessInstance(ProcessInstance processInstance) {
@ -1267,6 +1306,7 @@ public class ProcessService {
/**
* insert or update command
*
* @param command command
* @return save command result
*/
@ -1280,6 +1320,7 @@ public class ProcessService {
/**
* insert or update task instance
*
* @param taskInstance taskInstance
* @return save task instance result
*/
@ -1293,6 +1334,7 @@ public class ProcessService {
/**
* insert task instance
*
* @param taskInstance taskInstance
* @return create task instance result
*/
@ -1303,6 +1345,7 @@ public class ProcessService {
/**
* update task instance
*
* @param taskInstance taskInstance
* @return update task instance result
*/
@ -1313,6 +1356,7 @@ public class ProcessService {
/**
* delete a command by id
*
* @param id id
*/
public void delCommandById(int id) {
@ -1321,6 +1365,7 @@ public class ProcessService {
/**
* find task instance by id
*
* @param taskId task id
* @return task intance
*/
@ -1330,6 +1375,7 @@ public class ProcessService {
/**
* package task instanceassociate processInstance and processDefine
*
* @param taskInstId taskInstId
* @return task instance
*/
@ -1351,6 +1397,7 @@ public class ProcessService {
/**
* get id list by task state
*
* @param instanceId instanceId
* @param state state
* @return task instance states
@ -1361,6 +1408,7 @@ public class ProcessService {
/**
* find valid task list by process definition id
*
* @param processInstanceId processInstanceId
* @return task instance list
*/
@ -1370,6 +1418,7 @@ public class ProcessService {
/**
* find previous task list by work process id
*
* @param processInstanceId processInstanceId
* @return task instance list
*/
@ -1379,6 +1428,7 @@ public class ProcessService {
/**
* update work process instance map
*
* @param processInstanceMap processInstanceMap
* @return update process instance result
*/
@ -1388,6 +1438,7 @@ public class ProcessService {
/**
* create work process instance map
*
* @param processInstanceMap processInstanceMap
* @return create process instance result
*/
@ -1401,6 +1452,7 @@ public class ProcessService {
/**
* find work process map by parent process id and parent task id.
*
* @param parentWorkProcessId parentWorkProcessId
* @param parentTaskId parentTaskId
* @return process instance map
@ -1411,6 +1463,7 @@ public class ProcessService {
/**
* delete work process map by parent process id
*
* @param parentWorkProcessId parentWorkProcessId
* @return delete process map result
*/
@ -1421,6 +1474,7 @@ public class ProcessService {
/**
* find sub process instance
*
* @param parentProcessId parentProcessId
* @param parentTaskId parentTaskId
* @return process instance
@ -1437,6 +1491,7 @@ public class ProcessService {
/**
* find parent process instance
*
* @param subProcessId subProcessId
* @return process instance
*/
@ -1452,6 +1507,7 @@ public class ProcessService {
/**
* change task state
*
* @param state state
* @param startTime startTime
* @param host host
@ -1459,11 +1515,10 @@ public class ProcessService {
* @param logPath logPath
* @param taskInstId taskInstId
*/
public void changeTaskState(ExecutionStatus state, Date startTime, String host,
public void changeTaskState(TaskInstance taskInstance, ExecutionStatus state, Date startTime, String host,
String executePath,
String logPath,
int taskInstId) {
TaskInstance taskInstance = taskInstanceMapper.selectById(taskInstId);
taskInstance.setState(state);
taskInstance.setStartTime(startTime);
taskInstance.setHost(host);
@ -1474,6 +1529,7 @@ public class ProcessService {
/**
* update process instance
*
* @param processInstance processInstance
* @return update process instance result
*/
@ -1483,6 +1539,7 @@ public class ProcessService {
/**
* update the process instance
*
* @param processInstanceId processInstanceId
* @param processJson processJson
* @param globalParams globalParams
@ -1509,18 +1566,18 @@ public class ProcessService {
/**
* change task state
*
* @param state state
* @param endTime endTime
* @param taskInstId taskInstId
* @param varPool varPool
*/
public void changeTaskState(ExecutionStatus state,
public void changeTaskState(TaskInstance taskInstance, ExecutionStatus state,
Date endTime,
int processId,
String appIds,
int taskInstId,
String varPool) {
TaskInstance taskInstance = taskInstanceMapper.selectById(taskInstId);
taskInstance.setPid(processId);
taskInstance.setAppLink(appIds);
taskInstance.setState(state);
@ -1531,6 +1588,7 @@ public class ProcessService {
/**
* convert integer list to string list
*
* @param intList intList
* @return string list
*/
@ -1547,6 +1605,7 @@ public class ProcessService {
/**
* query schedule by id
*
* @param id id
* @return schedule
*/
@ -1556,6 +1615,7 @@ public class ProcessService {
/**
* query Schedule by processDefinitionId
*
* @param processDefinitionId processDefinitionId
* @see Schedule
*/
@ -1565,6 +1625,7 @@ public class ProcessService {
/**
* query need failover process instance
*
* @param host host
* @return process instance list
*/
@ -1574,6 +1635,7 @@ public class ProcessService {
/**
* process need failover process instance
*
* @param processInstance processInstance
*/
@Transactional(rollbackFor = RuntimeException.class)
@ -1593,6 +1655,7 @@ public class ProcessService {
/**
* query all need failover task instances by host
*
* @param host host
* @return task instance list
*/
@ -1603,6 +1666,7 @@ public class ProcessService {
/**
* find data source by id
*
* @param id id
* @return datasource
*/
@ -1612,6 +1676,7 @@ public class ProcessService {
/**
* update process instance state by id
*
* @param processInstanceId processInstanceId
* @param executionStatus executionStatus
* @return update process result
@ -1625,6 +1690,7 @@ public class ProcessService {
/**
* find process instance by the task id
*
* @param taskId taskId
* @return process instance
*/
@ -1638,6 +1704,7 @@ public class ProcessService {
/**
* find udf function list by id list string
*
* @param ids ids
* @return udf function list
*/
@ -1647,18 +1714,20 @@ public class ProcessService {
/**
* find tenant code by resource name
*
* @param resName resource name
* @param resourceType resource type
* @return tenant code
*/
public String queryTenantCodeByResName(String resName,ResourceType resourceType){
public String queryTenantCodeByResName(String resName, ResourceType resourceType) {
// in order to query tenant code successful although the version is older
String fullName = resName.startsWith("/") ? resName : String.format("/%s",resName);
String fullName = resName.startsWith("/") ? resName : String.format("/%s", resName);
return resourceMapper.queryTenantCodeByResourceName(fullName, resourceType.ordinal());
}
/**
* find schedule list by process define id.
*
* @param ids ids
* @return schedule list
*/
@ -1669,6 +1738,7 @@ public class ProcessService {
/**
* get dependency cycle by work process define id and scheduler fire time
*
* @param masterId masterId
* @param processDefinitionId processDefinitionId
* @param scheduledFireTime the time the task schedule is expected to trigger
@ -1676,22 +1746,23 @@ public class ProcessService {
* @throws Exception if error throws Exception
*/
public CycleDependency getCycleDependency(int masterId, int processDefinitionId, Date scheduledFireTime) throws Exception {
List<CycleDependency> list = getCycleDependencies(masterId,new int[]{processDefinitionId},scheduledFireTime);
List<CycleDependency> list = getCycleDependencies(masterId, new int[] {processDefinitionId}, scheduledFireTime);
return list.size() > 0 ? list.get(0) : null;
}
/**
* get dependency cycle list by work process define id list and scheduler fire time
*
* @param masterId masterId
* @param ids ids
* @param scheduledFireTime the time the task schedule is expected to trigger
* @return CycleDependency list
* @throws Exception if error throws Exception
*/
public List<CycleDependency> getCycleDependencies(int masterId,int[] ids,Date scheduledFireTime) throws Exception {
public List<CycleDependency> getCycleDependencies(int masterId, int[] ids, Date scheduledFireTime) throws Exception {
List<CycleDependency> cycleDependencyList = new ArrayList<CycleDependency>();
if(ArrayUtils.isEmpty(ids)){
if (null == ids || ids.length == 0) {
logger.warn("ids[] is empty!is invalid!");
return cycleDependencyList;
}
@ -1706,13 +1777,13 @@ public class ProcessService {
List<Date> list;
List<Schedule> schedules = this.selectAllByProcessDefineId(ids);
// for all scheduling information
for (Schedule depSchedule:schedules) {
for (Schedule depSchedule : schedules) {
strCrontab = depSchedule.getCrontab();
depCronExpression = CronUtils.parse2CronExpression(strCrontab);
depCron = CronUtils.parse2Cron(strCrontab);
CycleEnum cycleEnum = CronUtils.getMiniCycle(depCron);
if (cycleEnum == null) {
logger.error("{} is not valid",strCrontab);
logger.error("{} is not valid", strCrontab);
continue;
}
Calendar calendar = Calendar.getInstance();
@ -1720,16 +1791,16 @@ public class ProcessService {
/*case MINUTE:
calendar.add(Calendar.MINUTE,-61);*/
case HOUR:
calendar.add(Calendar.HOUR,-25);
calendar.add(Calendar.HOUR, -25);
break;
case DAY:
calendar.add(Calendar.DATE,-32);
calendar.add(Calendar.DATE, -32);
break;
case WEEK:
calendar.add(Calendar.DATE,-32);
calendar.add(Calendar.DATE, -32);
break;
case MONTH:
calendar.add(Calendar.MONTH,-13);
calendar.add(Calendar.MONTH, -13);
break;
default:
logger.warn("Dependent process definition's cycleEnum is {},not support!!", cycleEnum.name());
@ -1744,7 +1815,7 @@ public class ProcessService {
}
if (list.size() >= 1) {
start = list.get(list.size() - 1);
CycleDependency dependency = new CycleDependency(depSchedule.getProcessDefinitionId(),start, CronUtils.getExpirationTime(start, cycleEnum), cycleEnum);
CycleDependency dependency = new CycleDependency(depSchedule.getProcessDefinitionId(), start, CronUtils.getExpirationTime(start, cycleEnum), cycleEnum);
cycleDependencyList.add(dependency);
}
@ -1754,6 +1825,7 @@ public class ProcessService {
/**
* find last scheduler process instance in the date interval
*
* @param definitionId definitionId
* @param dateInterval dateInterval
* @return process instance
@ -1766,6 +1838,7 @@ public class ProcessService {
/**
* find last manual process instance interval
*
* @param definitionId process definition id
* @param dateInterval dateInterval
* @return process instance
@ -1778,6 +1851,7 @@ public class ProcessService {
/**
* find last running process instance
*
* @param definitionId process definition id
* @param startTime start time
* @param endTime end time
@ -1792,6 +1866,7 @@ public class ProcessService {
/**
* query user queue by process instance id
*
* @param processInstanceId processInstanceId
* @return queue
*/
@ -1811,6 +1886,7 @@ public class ProcessService {
/**
* get task worker group
*
* @param taskInstance taskInstance
* @return workerGroupId
*/
@ -1832,6 +1908,7 @@ public class ProcessService {
/**
* get have perm project list
*
* @param userId userId
* @return project list
*/
@ -1851,6 +1928,7 @@ public class ProcessService {
/**
* get have perm project ids
*
* @param userId userId
* @return project ids
*/
@ -1865,11 +1943,12 @@ public class ProcessService {
/**
* list unauthorized udf function
*
* @param userId user id
* @param needChecks data source id array
* @return unauthorized udf function list
*/
public <T> List<T> listUnauthorized(int userId,T[] needChecks,AuthorizationType authorizationType) {
public <T> List<T> listUnauthorized(int userId, T[] needChecks, AuthorizationType authorizationType) {
List<T> resultList = new ArrayList<T>();
if (Objects.nonNull(needChecks) && needChecks.length > 0) {
@ -1889,7 +1968,7 @@ public class ProcessService {
originResSet.removeAll(authorizedUdfFiles);
break;
case DATASOURCE:
Set<Integer> authorizedDatasources = dataSourceMapper.listAuthorizedDataSource(userId,needChecks).stream().map(t -> t.getId()).collect(toSet());
Set<Integer> authorizedDatasources = dataSourceMapper.listAuthorizedDataSource(userId, needChecks).stream().map(t -> t.getId()).collect(toSet());
originResSet.removeAll(authorizedDatasources);
break;
case UDF:
@ -1908,6 +1987,7 @@ public class ProcessService {
/**
* get user by user id
*
* @param userId user id
* @return User
*/
@ -1917,6 +1997,7 @@ public class ProcessService {
/**
* get resource by resoruce id
*
* @param resoruceId resource id
* @return Resource
*/
@ -1926,6 +2007,7 @@ public class ProcessService {
/**
* list resources by ids
*
* @param resIds resIds
* @return resource list
*/
@ -1935,6 +2017,7 @@ public class ProcessService {
/**
* format task app id in task instance
*
* @param taskInstance
* @return
*/

1
pom.xml

@ -835,6 +835,7 @@
<include>**/server/master/SubProcessTaskTest.java</include>
<include>**/server/master/processor/TaskAckProcessorTest.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/utils/DataxUtilsTest.java</include>
<include>**/server/utils/ExecutionContextTestUtils.java</include>

Loading…
Cancel
Save