Browse Source

优化worker server获取任务的性能

pull/2/head
lidongdai 5 years ago
parent
commit
c8906686d4
  1. 6
      escheduler-alert/src/main/resources/alert.properties
  2. 8
      escheduler-common/src/main/java/cn/escheduler/common/Constants.java
  3. 9
      escheduler-common/src/main/java/cn/escheduler/common/queue/ITaskQueue.java
  4. 2
      escheduler-common/src/main/java/cn/escheduler/common/queue/TaskQueueFactory.java
  5. 100
      escheduler-common/src/main/java/cn/escheduler/common/queue/TaskQueueZkImpl.java
  6. 6
      escheduler-common/src/main/java/cn/escheduler/common/zk/AbstractZKClient.java
  7. 6
      escheduler-common/src/test/java/cn/escheduler/common/os/OSUtilsTest.java
  8. 8
      escheduler-common/src/test/java/cn/escheduler/common/queue/TaskQueueImplTest.java
  9. 74
      escheduler-dao/src/main/java/cn/escheduler/dao/ProcessDao.java
  10. 163
      escheduler-server/src/main/java/cn/escheduler/server/worker/runner/FetchTaskThread.java

6
escheduler-alert/src/main/resources/alert.properties

@ -3,10 +3,10 @@ alert.type=EMAIL
# mail server configuration # mail server configuration
mail.protocol=SMTP mail.protocol=SMTP
mail.server.host=smtp.exmail.qq.com mail.server.host=smtp.163.com
mail.server.port=25 mail.server.port=25
mail.sender=xxxxxxx mail.sender=d66380022@163.com
mail.passwd=xxxxxxx mail.passwd=qwertyuiop123
#xls file path,need create if not exist #xls file path,need create if not exist
xls.file.path=/opt/xls xls.file.path=/opt/xls

8
escheduler-common/src/main/java/cn/escheduler/common/Constants.java

@ -192,6 +192,11 @@ public final class Constants {
*/ */
public static final String SEMICOLON = ";"; public static final String SEMICOLON = ";";
/**
* DOT .
*/
public static final String DOT = ".";
/** /**
* ZOOKEEPER_SESSION_TIMEOUT * ZOOKEEPER_SESSION_TIMEOUT
*/ */
@ -822,6 +827,7 @@ public final class Constants {
/** /**
* * default worker group id
*/ */
public static final int DEFAULT_WORKER_ID = -1;
} }

9
escheduler-common/src/main/java/cn/escheduler/common/queue/ITaskQueue.java

@ -24,20 +24,17 @@ public interface ITaskQueue {
/** /**
* take out all the elements * take out all the elements
* *
* this method has deprecated
* use checkTaskExists instead
* *
* @param key * @param key
* @return * @return
*/ */
@Deprecated
List<String> getAllTasks(String key); List<String> getAllTasks(String key);
/** /**
* check task exists in the task queue or not * check task exists in the task queue or not
* *
* @param key queue name * @param key queue name
* @param task ${priority}_${processInstanceId}_${taskId} * @param task ${processInstancePriority}_${processInstanceId}_${taskInstancePriority}_${taskId}
* @return true if exists in the queue * @return true if exists in the queue
*/ */
boolean checkTaskExists(String key, String task); boolean checkTaskExists(String key, String task);
@ -54,10 +51,10 @@ public interface ITaskQueue {
* an element pops out of the queue * an element pops out of the queue
* *
* @param key queue name * @param key queue name
* @param remove whether remove the element * @param n how many elements to poll
* @return * @return
*/ */
String poll(String key, boolean remove); List<String> poll(String key, int n);
/** /**
* remove a element from queue * remove a element from queue

2
escheduler-common/src/main/java/cn/escheduler/common/queue/TaskQueueFactory.java

@ -42,7 +42,7 @@ public class TaskQueueFactory {
public static ITaskQueue getTaskQueueInstance() { public static ITaskQueue getTaskQueueInstance() {
String queueImplValue = CommonUtils.getQueueImplValue(); String queueImplValue = CommonUtils.getQueueImplValue();
if (StringUtils.isNotBlank(queueImplValue)) { if (StringUtils.isNotBlank(queueImplValue)) {
// queueImplValue = StringUtils.trim(queueImplValue); // queueImplValue = IpUtils.trim(queueImplValue);
// if (SCHEDULER_QUEUE_REDIS_IMPL.equals(queueImplValue)) { // if (SCHEDULER_QUEUE_REDIS_IMPL.equals(queueImplValue)) {
// logger.info("task queue impl use reids "); // logger.info("task queue impl use reids ");

100
escheduler-common/src/main/java/cn/escheduler/common/queue/TaskQueueZkImpl.java

@ -19,6 +19,8 @@ package cn.escheduler.common.queue;
import cn.escheduler.common.Constants; import cn.escheduler.common.Constants;
import cn.escheduler.common.utils.Bytes; import cn.escheduler.common.utils.Bytes;
import cn.escheduler.common.utils.IpUtils;
import cn.escheduler.common.utils.OSUtils;
import cn.escheduler.common.zk.AbstractZKClient; import cn.escheduler.common.zk.AbstractZKClient;
import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFramework;
import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.CreateMode;
@ -26,10 +28,7 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.ArrayList; import java.util.*;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
/** /**
* A singleton of a task queue implemented with zookeeper * A singleton of a task queue implemented with zookeeper
@ -62,7 +61,6 @@ public class TaskQueueZkImpl extends AbstractZKClient implements ITaskQueue {
* @param key task queue name * @param key task queue name
* @return * @return
*/ */
@Deprecated
@Override @Override
public List<String> getAllTasks(String key) { public List<String> getAllTasks(String key) {
try { try {
@ -80,7 +78,7 @@ public class TaskQueueZkImpl extends AbstractZKClient implements ITaskQueue {
* check task exists in the task queue or not * check task exists in the task queue or not
* *
* @param key queue name * @param key queue name
* @param task ${priority}_${processInstanceId}_${taskId} * @param task ${processInstancePriority}_${processInstanceId}_${taskInstancePriority}_${taskId}
* @return true if exists in the queue * @return true if exists in the queue
*/ */
@Override @Override
@ -110,7 +108,7 @@ public class TaskQueueZkImpl extends AbstractZKClient implements ITaskQueue {
* add task to tasks queue * add task to tasks queue
* *
* @param key task queue name * @param key task queue name
* @param value ${priority}_${processInstanceId}_${taskId} * @param value ${processInstancePriority}_${processInstanceId}_${taskInstancePriority}_${taskId}_host1,host2,...
*/ */
@Override @Override
public void add(String key, String value) { public void add(String key, String value) {
@ -129,16 +127,16 @@ public class TaskQueueZkImpl extends AbstractZKClient implements ITaskQueue {
/** /**
* An element pops out of the queue <p> * An element pops out of the queue <p>
* note: * note:
* ${processInstancePriority}_${processInstanceId}_${taskInstancePriority}_${taskId} * ${processInstancePriority}_${processInstanceId}_${taskInstancePriority}_${taskId}_host1,host2,...
* The tasks with the highest priority are selected by comparing the priorities of the above four levels from high to low. * The tasks with the highest priority are selected by comparing the priorities of the above four levels from high to low.
* *
* 流程实例优先级_流程实例id_任务优先级_任务id high <- low * 流程优先级_流程实例id_任务优先级_任务id_任务执行的机器id1,任务执行的机器id2,... high <- low
* @param key task queue name * @param key task queue name
* @param remove whether remove the element * @param tasksNum how many elements to poll
* @return the task id to be executed * @return the task ids to be executed
*/ */
@Override @Override
public String poll(String key, boolean remove) { public List<String> poll(String key, int tasksNum) {
try{ try{
CuratorFramework zk = getZkClient(); CuratorFramework zk = getZkClient();
String tasksQueuePath = getTasksPath(key) + Constants.SINGLE_SLASH; String tasksQueuePath = getTasksPath(key) + Constants.SINGLE_SLASH;
@ -146,55 +144,79 @@ public class TaskQueueZkImpl extends AbstractZKClient implements ITaskQueue {
if(list != null && list.size() > 0){ if(list != null && list.size() > 0){
String workerIp = OSUtils.getHost();
String workerIpLongStr = String.valueOf(IpUtils.ipToLong(workerIp));
int size = list.size(); int size = list.size();
String formatTargetTask = null;
String targetTaskKey = null; Set<String> taskTreeSet = new TreeSet<>();
for (int i = 0; i < size; i++) { for (int i = 0; i < size; i++) {
String taskDetail = list.get(i); String taskDetail = list.get(i);
String[] taskDetailArrs = taskDetail.split(Constants.UNDERLINE); String[] taskDetailArrs = taskDetail.split(Constants.UNDERLINE);
if(taskDetailArrs.length == 4){ //向前版本兼容
if(taskDetailArrs.length >= 4){
//format ${processInstancePriority}_${processInstanceId}_${taskInstancePriority}_${taskId} //format ${processInstancePriority}_${processInstanceId}_${taskInstancePriority}_${taskId}
String formatTask = String.format("%s_%010d_%s_%010d", taskDetailArrs[0], Long.parseLong(taskDetailArrs[1]), taskDetailArrs[2], Long.parseLong(taskDetailArrs[3])); String formatTask = String.format("%s_%010d_%s_%010d", taskDetailArrs[0], Long.parseLong(taskDetailArrs[1]), taskDetailArrs[2], Long.parseLong(taskDetailArrs[3]));
if(i > 0){ if(taskDetailArrs.length > 4){
int result = formatTask.compareTo(formatTargetTask); String taskHosts = taskDetailArrs[4];
if(result < 0){
formatTargetTask = formatTask; //task can assign to any worker host if equals default ip value of worker server
targetTaskKey = taskDetail; if(!taskHosts.equals(Constants.DEFAULT_WORKER_ID)){
String[] taskHostsArr = taskHosts.split(Constants.COMMA);
if(!Arrays.asList(taskHostsArr).contains(workerIpLongStr)){
continue;
}
} }
}else{
formatTargetTask = formatTask;
targetTaskKey = taskDetail;
} }
}else{
logger.error("task queue poll error, task detail :{} , please check!", taskDetail); taskTreeSet.add(formatTask);
} }
}
if(formatTargetTask != null){ }
String taskIdPath = tasksQueuePath + targetTaskKey;
logger.info("consume task {}", taskIdPath); List<String> taskslist = getTasksListFromTreeSet(tasksNum, taskTreeSet);
String[] vals = targetTaskKey.split(Constants.UNDERLINE); logger.info("consume tasks: {},there still have {} tasks need to be executed", Arrays.toString(taskslist.toArray()), size - taskslist.size());
if(remove){ return taskslist;
removeNode(key, targetTaskKey); }else{
} Thread.sleep(Constants.SLEEP_TIME_MILLIS);
logger.info("consume task: {},there still have {} tasks need to be executed", vals[vals.length - 1], size - 1);
return targetTaskKey;
}else{
logger.error("should not go here, task queue poll error, please check!");
}
} }
} catch (Exception e) { } catch (Exception e) {
logger.error("add task to tasks queue exception",e); logger.error("add task to tasks queue exception",e);
} }
return null; return new ArrayList<String>();
}
/**
* get task list from tree set
*
* @param tasksNum
* @param taskTreeSet
*/
public List<String> getTasksListFromTreeSet(int tasksNum, Set<String> taskTreeSet) {
Iterator<String> iterator = taskTreeSet.iterator();
int j = 0;
List<String> taskslist = new ArrayList<>(tasksNum);
while(iterator.hasNext()){
if(j++ < tasksNum){
String task = iterator.next();
taskslist.add(task);
}
}
return taskslist;
} }
@Override @Override
public void removeNode(String key, String nodeValue){ public void removeNode(String key, String nodeValue){

6
escheduler-common/src/main/java/cn/escheduler/common/zk/AbstractZKClient.java

@ -312,7 +312,11 @@ public abstract class AbstractZKClient {
childrenList = zkClient.getChildren().forPath(masterZNodeParentPath); childrenList = zkClient.getChildren().forPath(masterZNodeParentPath);
} }
} catch (Exception e) { } catch (Exception e) {
logger.warn(e.getMessage(),e); // logger.warn(e.getMessage());
if(!e.getMessage().contains("java.lang.IllegalStateException: instance must be started")){
logger.warn(e.getMessage(),e);
}
return childrenList.size(); return childrenList.size();
} }
return childrenList.size(); return childrenList.size();

6
escheduler-common/src/test/java/cn/escheduler/common/os/OSUtilsTest.java

@ -37,6 +37,12 @@ public class OSUtilsTest {
// static HardwareAbstractionLayer hal = si.getHardware(); // static HardwareAbstractionLayer hal = si.getHardware();
@Test
public void getHost(){
logger.info(OSUtils.getHost());
}
@Test @Test
public void memoryUsage() { public void memoryUsage() {
logger.info("memoryUsage : {}", OSUtils.memoryUsage());// 0.3361799418926239 logger.info("memoryUsage : {}", OSUtils.memoryUsage());// 0.3361799418926239

8
escheduler-common/src/test/java/cn/escheduler/common/queue/TaskQueueImplTest.java

@ -49,9 +49,11 @@ public class TaskQueueImplTest {
tasksQueue.add(Constants.SCHEDULER_TASKS_QUEUE,"4"); tasksQueue.add(Constants.SCHEDULER_TASKS_QUEUE,"4");
//pop //pop
String node1 = tasksQueue.poll(Constants.SCHEDULER_TASKS_QUEUE, false); String node1 = tasksQueue.poll(Constants.SCHEDULER_TASKS_QUEUE, 1).get(0);
assertEquals(node1,"1"); assertEquals(node1,"1");
String node2 = tasksQueue.poll(Constants.SCHEDULER_TASKS_QUEUE, false);
String node2 = tasksQueue.poll(Constants.SCHEDULER_TASKS_QUEUE, 1).get(0);
assertEquals(node2,"2"); assertEquals(node2,"2");
//sadd //sadd
@ -99,7 +101,7 @@ public class TaskQueueImplTest {
} }
} }
String node1 = tasksQueue.poll(Constants.SCHEDULER_TASKS_QUEUE, false); String node1 = tasksQueue.poll(Constants.SCHEDULER_TASKS_QUEUE, 1).get(0);
assertEquals(node1,"0"); assertEquals(node1,"0");
//clear all data //clear all data

74
escheduler-dao/src/main/java/cn/escheduler/dao/ProcessDao.java

@ -24,6 +24,7 @@ import cn.escheduler.common.queue.ITaskQueue;
import cn.escheduler.common.queue.TaskQueueFactory; import cn.escheduler.common.queue.TaskQueueFactory;
import cn.escheduler.common.task.subprocess.SubProcessParameters; import cn.escheduler.common.task.subprocess.SubProcessParameters;
import cn.escheduler.common.utils.DateUtils; import cn.escheduler.common.utils.DateUtils;
import cn.escheduler.common.utils.IpUtils;
import cn.escheduler.common.utils.JSONUtils; import cn.escheduler.common.utils.JSONUtils;
import cn.escheduler.common.utils.ParameterUtils; import cn.escheduler.common.utils.ParameterUtils;
import cn.escheduler.dao.mapper.*; import cn.escheduler.dao.mapper.*;
@ -108,7 +109,7 @@ public class ProcessDao extends AbstractBaseDao {
*/ */
@Override @Override
protected void init() { protected void init() {
userMapper=getMapper(UserMapper.class); userMapper = getMapper(UserMapper.class);
processDefineMapper = getMapper(ProcessDefinitionMapper.class); processDefineMapper = getMapper(ProcessDefinitionMapper.class);
processInstanceMapper = getMapper(ProcessInstanceMapper.class); processInstanceMapper = getMapper(ProcessInstanceMapper.class);
dataSourceMapper = getMapper(DataSourceMapper.class); dataSourceMapper = getMapper(DataSourceMapper.class);
@ -947,11 +948,58 @@ public class ProcessDao extends AbstractBaseDao {
* *
* 流程实例优先级_流程实例id_任务优先级_任务id high <- low * 流程实例优先级_流程实例id_任务优先级_任务id high <- low
* *
* @param task * @param taskInstance
* @return * @return
*/ */
private String taskZkInfo(TaskInstance task) { private String taskZkInfo(TaskInstance taskInstance) {
return String.valueOf(task.getProcessInstancePriority().ordinal()) + Constants.UNDERLINE + task.getProcessInstanceId() + Constants.UNDERLINE + task.getTaskInstancePriority().ordinal() + Constants.UNDERLINE + task.getId();
int taskWorkerGroupId = getTaskWorkerGroupId(taskInstance);
StringBuilder sb = new StringBuilder(100);
sb.append(taskInstance.getProcessInstancePriority().ordinal()).append(Constants.UNDERLINE)
.append(taskInstance.getProcessInstanceId()).append(Constants.UNDERLINE)
.append(taskInstance.getTaskInstancePriority().ordinal()).append(Constants.UNDERLINE)
.append(taskInstance.getId()).append(Constants.UNDERLINE);
if(taskWorkerGroupId > 0){
//not to find data from db
WorkerGroup workerGroup = queryWorkerGroupById(taskWorkerGroupId);
if(workerGroup == null ){
logger.info("task {} cannot find the worker group, use all worker instead.", taskInstance.getId());
sb.append(Constants.DEFAULT_WORKER_ID);
return sb.toString();
}
String ips = workerGroup.getIpList();
if(StringUtils.isBlank(ips)){
logger.error("task:{} worker group:{} parameters(ip_list) is null, this task would be running on all workers",
taskInstance.getId(), workerGroup.getId());
sb.append(Constants.DEFAULT_WORKER_ID);
return sb.toString();
}
StringBuilder ipSb = new StringBuilder(100);
String[] ipArray = ips.split(COMMA);
for (String ip : ipArray) {
long ipLong = IpUtils.ipToLong(ip);
ipSb.append(ipLong).append(COMMA);
}
if(ipSb.length() > 0) {
ipSb.deleteCharAt(ipSb.length() - 1);
}
sb.append(ipSb);
}else{
sb.append(Constants.DEFAULT_WORKER_ID);
}
return sb.toString();
} }
/** /**
@ -1591,5 +1639,23 @@ public class ProcessDao extends AbstractBaseDao {
} }
/**
* get task worker group id
*
* @param taskInstance
* @return
*/
public int getTaskWorkerGroupId(TaskInstance taskInstance) {
int taskWorkerGroupId = taskInstance.getWorkerGroupId();
ProcessInstance processInstance = findProcessInstanceByTaskId(taskInstance.getId());
if(processInstance == null){
logger.error("cannot find the task:{} process instance", taskInstance.getId());
}
int processWorkerGroupId = processInstance.getWorkerGroupId();
taskWorkerGroupId = (taskWorkerGroupId <= 0 ? processWorkerGroupId : taskWorkerGroupId);
return taskWorkerGroupId;
}
} }

163
escheduler-server/src/main/java/cn/escheduler/server/worker/runner/FetchTaskThread.java

@ -28,8 +28,9 @@ import cn.escheduler.dao.model.ProcessInstance;
import cn.escheduler.dao.model.TaskInstance; import cn.escheduler.dao.model.TaskInstance;
import cn.escheduler.dao.model.WorkerGroup; import cn.escheduler.dao.model.WorkerGroup;
import cn.escheduler.server.zk.ZKWorkerClient; import cn.escheduler.server.zk.ZKWorkerClient;
import com.cronutils.utils.StringUtils;
import org.apache.commons.configuration.Configuration; import org.apache.commons.configuration.Configuration;
import org.apache.commons.lang3.StringUtils;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.locks.InterProcessMutex; import org.apache.curator.framework.recipes.locks.InterProcessMutex;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -101,15 +102,7 @@ public class FetchTaskThread implements Runnable{
*/ */
private boolean checkWorkerGroup(TaskInstance taskInstance, String host){ private boolean checkWorkerGroup(TaskInstance taskInstance, String host){
int taskWorkerGroupId = taskInstance.getWorkerGroupId(); int taskWorkerGroupId = processDao.getTaskWorkerGroupId(taskInstance);
ProcessInstance processInstance = processDao.findProcessInstanceByTaskId(taskInstance.getId());
if(processInstance == null){
logger.error("cannot find the task:{} process instance", taskInstance.getId());
return false;
}
int processWorkerGroupId = processInstance.getWorkerGroupId();
taskWorkerGroupId = (taskWorkerGroupId <= 0 ? processWorkerGroupId : taskWorkerGroupId);
if(taskWorkerGroupId <= 0){ if(taskWorkerGroupId <= 0){
return true; return true;
@ -120,118 +113,124 @@ public class FetchTaskThread implements Runnable{
return true; return true;
} }
String ips = workerGroup.getIpList(); String ips = workerGroup.getIpList();
if(ips == null){ if(StringUtils.isBlank(ips)){
logger.error("task:{} worker group:{} parameters(ip_list) is null, this task would be running on all workers", logger.error("task:{} worker group:{} parameters(ip_list) is null, this task would be running on all workers",
taskInstance.getId(), workerGroup.getId()); taskInstance.getId(), workerGroup.getId());
} }
String[] ipArray = ips.split(","); String[] ipArray = ips.split(Constants.COMMA);
List<String> ipList = Arrays.asList(ipArray); List<String> ipList = Arrays.asList(ipArray);
return ipList.contains(host); return ipList.contains(host);
} }
@Override @Override
public void run() { public void run() {
while (Stopper.isRunning()){ while (Stopper.isRunning()){
InterProcessMutex mutex = null; InterProcessMutex mutex = null;
try { try {
if(OSUtils.checkResource(this.conf, false)) {
// creating distributed locks, lock path /escheduler/lock/worker ThreadPoolExecutor poolExecutor = (ThreadPoolExecutor) workerExecService;
String zNodeLockPath = zkWorkerClient.getWorkerLockPath();
mutex = new InterProcessMutex(zkWorkerClient.getZkClient(), zNodeLockPath);
mutex.acquire();
ThreadPoolExecutor poolExecutor = (ThreadPoolExecutor) workerExecService; //check memory and cpu usage and threads
if(OSUtils.checkResource(this.conf, false) && checkThreadCount(poolExecutor)) {
for (int i = 0; i < taskNum; i++) { //whether have tasks, if no tasks , no need lock //get all tasks
List<String> tasksQueueList = taskQueue.getAllTasks(Constants.SCHEDULER_TASKS_QUEUE);
int activeCount = poolExecutor.getActiveCount(); if(tasksQueueList.size() > 0){
if (activeCount >= workerExecNums) { // creating distributed locks, lock path /escheduler/lock/worker
logger.info("thread insufficient , activeCount : {} , workerExecNums : {}",activeCount,workerExecNums); String zNodeLockPath = zkWorkerClient.getWorkerLockPath();
continue; mutex = new InterProcessMutex(zkWorkerClient.getZkClient(), zNodeLockPath);
} mutex.acquire();
// task instance id str // task instance id str
String taskQueueStr = taskQueue.poll(Constants.SCHEDULER_TASKS_QUEUE, false); List<String> taskQueueStrArr = taskQueue.poll(Constants.SCHEDULER_TASKS_QUEUE, taskNum);
if (!StringUtils.isEmpty(taskQueueStr )) { for(String taskQueueStr : taskQueueStrArr){
if (StringUtils.isNotBlank(taskQueueStr )) {
String[] taskStringArray = taskQueueStr.split(Constants.UNDERLINE); if (!checkThreadCount(poolExecutor)) {
String taskInstIdStr = taskStringArray[taskStringArray.length - 1]; break;
Date now = new Date(); }
Integer taskId = Integer.parseInt(taskInstIdStr);
// find task instance by task id String[] taskStringArray = taskQueueStr.split(Constants.UNDERLINE);
TaskInstance taskInstance = processDao.findTaskInstanceById(taskId); String taskInstIdStr = taskStringArray[taskStringArray.length - 1];
Date now = new Date();
Integer taskId = Integer.parseInt(taskInstIdStr);
logger.info("worker fetch taskId : {} from queue ", taskId); // find task instance by task id
TaskInstance taskInstance = processDao.findTaskInstanceById(taskId);
int retryTimes = 30; logger.info("worker fetch taskId : {} from queue ", taskId);
// mainly to wait for the master insert task to succeed
while (taskInstance == null && retryTimes > 0) {
Thread.sleep(Constants.SLEEP_TIME_MILLIS);
taskInstance = processDao.findTaskInstanceById(taskId);
retryTimes--;
}
if (taskInstance == null ) { int retryTimes = 30;
logger.error("task instance is null. task id : {} ", taskId); // mainly to wait for the master insert task to succeed
continue; while (taskInstance == null && retryTimes > 0) {
} Thread.sleep(Constants.SLEEP_TIME_MILLIS);
if(!checkWorkerGroup(taskInstance, OSUtils.getHost())){ taskInstance = processDao.findTaskInstanceById(taskId);
continue; retryTimes--;
} }
taskQueue.removeNode(Constants.SCHEDULER_TASKS_QUEUE, taskQueueStr);
logger.info("remove task:{} from queue", taskQueueStr); if (taskInstance == null ) {
logger.error("task instance is null. task id : {} ", taskId);
continue;
}
// set execute task worker host if(!checkWorkerGroup(taskInstance, OSUtils.getHost())){
taskInstance.setHost(OSUtils.getHost()); continue;
taskInstance.setStartTime(now); }
taskQueue.removeNode(Constants.SCHEDULER_TASKS_QUEUE, taskQueueStr);
logger.info("remove task:{} from queue", taskQueueStr);
// set execute task worker host
taskInstance.setHost(OSUtils.getHost());
taskInstance.setStartTime(now);
// get process instance
ProcessInstance processInstance = processDao.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
// get process define // get process instance
ProcessDefinition processDefine = processDao.findProcessDefineById(taskInstance.getProcessDefinitionId()); ProcessInstance processInstance = processDao.findProcessInstanceDetailById(taskInstance.getProcessInstanceId());
// get process define
ProcessDefinition processDefine = processDao.findProcessDefineById(taskInstance.getProcessDefinitionId());
taskInstance.setProcessInstance(processInstance);
taskInstance.setProcessDefine(processDefine);
taskInstance.setProcessInstance(processInstance);
taskInstance.setProcessDefine(processDefine);
// get local execute path
String execLocalPath = FileUtils.getProcessExecDir(processDefine.getProjectId(),
processDefine.getId(),
processInstance.getId(),
taskInstance.getId());
logger.info("task instance local execute path : {} ", execLocalPath);
// get local execute path
String execLocalPath = FileUtils.getProcessExecDir(processDefine.getProjectId(),
processDefine.getId(),
processInstance.getId(),
taskInstance.getId());
logger.info("task instance local execute path : {} ", execLocalPath);
// set task execute path
taskInstance.setExecutePath(execLocalPath);
// check and create Linux users // set task execute path
FileUtils.createWorkDirAndUserIfAbsent(execLocalPath, taskInstance.setExecutePath(execLocalPath);
processInstance.getTenantCode(), logger);
logger.info("task : {} ready to submit to task scheduler thread",taskId); // check and create Linux users
// submit task FileUtils.createWorkDirAndUserIfAbsent(execLocalPath,
workerExecService.submit(new TaskScheduleThread(taskInstance, processDao)); processInstance.getTenantCode(), logger);
logger.info("task : {} ready to submit to task scheduler thread",taskId);
// submit task
workerExecService.submit(new TaskScheduleThread(taskInstance, processDao));
}
} }
} }
} }
Thread.sleep(Constants.SLEEP_TIME_MILLIS); Thread.sleep(Constants.SLEEP_TIME_MILLIS);
}catch (Exception e){ }catch (Exception e){
logger.error("fetch task thread exception : " + e.getMessage(),e); logger.error("fetch task thread exception : " + e.getMessage(),e);
} }finally {
finally {
if (mutex != null){ if (mutex != null){
try { try {
mutex.release(); mutex.release();
@ -246,4 +245,18 @@ public class FetchTaskThread implements Runnable{
} }
} }
} }
/**
*
* @param poolExecutor
* @return
*/
private boolean checkThreadCount(ThreadPoolExecutor poolExecutor) {
int activeCount = poolExecutor.getActiveCount();
if (activeCount >= workerExecNums) {
logger.info("thread insufficient , activeCount : {} , workerExecNums : {}, will sleep : {} millis for thread resource", activeCount, workerExecNums, Constants.SLEEP_TIME_MILLIS);
return false;
}
return true;
}
} }
Loading…
Cancel
Save