分布式调度框架。
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

1237 lines
44 KiB

/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.server.master.runner;
import com.alibaba.fastjson.JSON;
import com.google.common.collect.Lists;
merge dev-db to dev (#1426) * [dolphinscheduler-1345] [newfeature] Add DB2 Datasource (#1391) * Fix the problem that the 'queueId' is not present when creating a tenant based on the default queue. (#1409) * dolphinscheduler-#1403][bug]improve the check rules (#1408) 1. When check failed, we don’t know whitch parameter has is wrong, Because username、password、email and phone were checks together. I refactored the check method ,Now it will return failed msg by each field. 2. The email check regex support [_|\-|\.]?) in createUser.vue, But it do not support in backend server , I fix it, Now they have the same check regex both in frontend and backend * jcip-annotations define version information and maven-assembly-plugin add groupId (#1413) * "v-for" add key (#1419) * [dolphinscheduler-#1397] [bug]Resources can not be previewed or updated  (#1406) When create an resource the name will add the suffix, But When rename the resource there is no suffix add to the name, So When update resource name without suffix just like "test.sh" => "test" , Then the bug reproduced. To fix this bug i add the logic bellow: When rename, if the name without suffix then add it ,else use the origin name * simply server module configs (#1424) * move updateTaskState into try/catch block in case of exception * fix NPE * using conf.getInt instead of getString * for AbstractZKClient, remove the log, for it will print the same log message in createZNodePath. for AlertDao, correct the spelling. * duplicate * refactor getTaskWorkerGroupId * add friendly log * update hearbeat thread num = 1 * fix the bug when worker execute task using queue. and remove checking Tenant user anymore in TaskScheduleThread * 1. move verifyTaskInstanceIsNull after taskInstance 2. keep verifyTenantIsNull/verifyTaskInstanceIsNull clean and readable * fix the message * delete before check to avoid KeeperException$NoNodeException * fix the message * check processInstance state before delete tenant * check processInstance state before delete worker group * refactor * merge api constants into common constatns * update the resource perm * update the dataSource perm * fix CheckUtils.checkUserParams method * update AlertGroupService, extends from BaseService, remove duplicate methods * refactor * modify method name * add hasProjectAndPerm method * using checkProject instead of getResultStatus * delete checkAuth method, using hasProjectAndPerm instead. * correct spelling * add transactional for deleteWorkerGroupById * add Transactional for deleteProcessInstanceById method * change sqlSessionTemplate singleton * change sqlSessionTemplate singleton and reformat code * fix unsuitable error message * update shutdownhook methods * fix worker log bug * fix api server debug mode bug * upgrade zk version * delete this line ,for zkClient.close() will do the whole thing * fix master server shutdown error * degrade zk version and add FourLetterWordMain class * fix PathChildrenCache not close * add Transactional for createSession method * add more message for java-doc * delete App, let spring manage connectionFactory * add license * add class Application for test support * refactor masterServer and workerServer * add args * fix the spring transaction not work bug * remove author * delete @Bean annotation * delete master/worker properties * updates * rename application.properties to application-dao.properties * delete this class * delete master/worker properties and refactory master/worker * delete unused imports * merge * delete unused config
5 years ago
import org.apache.commons.io.FileUtils;
import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.enums.*;
import org.apache.dolphinscheduler.common.graph.DAG;
import org.apache.dolphinscheduler.common.model.TaskNode;
import org.apache.dolphinscheduler.common.model.TaskNodeRelation;
import org.apache.dolphinscheduler.common.process.ProcessDag;
import org.apache.dolphinscheduler.common.task.conditions.ConditionsParameters;
import org.apache.dolphinscheduler.common.thread.Stopper;
import org.apache.dolphinscheduler.common.thread.ThreadUtils;
merge dev-db to dev (#1426) * [dolphinscheduler-1345] [newfeature] Add DB2 Datasource (#1391) * Fix the problem that the 'queueId' is not present when creating a tenant based on the default queue. (#1409) * dolphinscheduler-#1403][bug]improve the check rules (#1408) 1. When check failed, we don’t know whitch parameter has is wrong, Because username、password、email and phone were checks together. I refactored the check method ,Now it will return failed msg by each field. 2. The email check regex support [_|\-|\.]?) in createUser.vue, But it do not support in backend server , I fix it, Now they have the same check regex both in frontend and backend * jcip-annotations define version information and maven-assembly-plugin add groupId (#1413) * "v-for" add key (#1419) * [dolphinscheduler-#1397] [bug]Resources can not be previewed or updated  (#1406) When create an resource the name will add the suffix, But When rename the resource there is no suffix add to the name, So When update resource name without suffix just like "test.sh" => "test" , Then the bug reproduced. To fix this bug i add the logic bellow: When rename, if the name without suffix then add it ,else use the origin name * simply server module configs (#1424) * move updateTaskState into try/catch block in case of exception * fix NPE * using conf.getInt instead of getString * for AbstractZKClient, remove the log, for it will print the same log message in createZNodePath. for AlertDao, correct the spelling. * duplicate * refactor getTaskWorkerGroupId * add friendly log * update hearbeat thread num = 1 * fix the bug when worker execute task using queue. and remove checking Tenant user anymore in TaskScheduleThread * 1. move verifyTaskInstanceIsNull after taskInstance 2. keep verifyTenantIsNull/verifyTaskInstanceIsNull clean and readable * fix the message * delete before check to avoid KeeperException$NoNodeException * fix the message * check processInstance state before delete tenant * check processInstance state before delete worker group * refactor * merge api constants into common constatns * update the resource perm * update the dataSource perm * fix CheckUtils.checkUserParams method * update AlertGroupService, extends from BaseService, remove duplicate methods * refactor * modify method name * add hasProjectAndPerm method * using checkProject instead of getResultStatus * delete checkAuth method, using hasProjectAndPerm instead. * correct spelling * add transactional for deleteWorkerGroupById * add Transactional for deleteProcessInstanceById method * change sqlSessionTemplate singleton * change sqlSessionTemplate singleton and reformat code * fix unsuitable error message * update shutdownhook methods * fix worker log bug * fix api server debug mode bug * upgrade zk version * delete this line ,for zkClient.close() will do the whole thing * fix master server shutdown error * degrade zk version and add FourLetterWordMain class * fix PathChildrenCache not close * add Transactional for createSession method * add more message for java-doc * delete App, let spring manage connectionFactory * add license * add class Application for test support * refactor masterServer and workerServer * add args * fix the spring transaction not work bug * remove author * delete @Bean annotation * delete master/worker properties * updates * rename application.properties to application-dao.properties * delete this class * delete master/worker properties and refactory master/worker * delete unused imports * merge * delete unused config
5 years ago
import org.apache.dolphinscheduler.common.utils.*;
import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
import org.apache.dolphinscheduler.dao.entity.Schedule;
import org.apache.dolphinscheduler.dao.entity.TaskInstance;
import org.apache.dolphinscheduler.dao.utils.DagHelper;
import org.apache.dolphinscheduler.remote.NettyRemotingClient;
merge dev-db to dev (#1426) * [dolphinscheduler-1345] [newfeature] Add DB2 Datasource (#1391) * Fix the problem that the 'queueId' is not present when creating a tenant based on the default queue. (#1409) * dolphinscheduler-#1403][bug]improve the check rules (#1408) 1. When check failed, we don’t know whitch parameter has is wrong, Because username、password、email and phone were checks together. I refactored the check method ,Now it will return failed msg by each field. 2. The email check regex support [_|\-|\.]?) in createUser.vue, But it do not support in backend server , I fix it, Now they have the same check regex both in frontend and backend * jcip-annotations define version information and maven-assembly-plugin add groupId (#1413) * "v-for" add key (#1419) * [dolphinscheduler-#1397] [bug]Resources can not be previewed or updated  (#1406) When create an resource the name will add the suffix, But When rename the resource there is no suffix add to the name, So When update resource name without suffix just like "test.sh" => "test" , Then the bug reproduced. To fix this bug i add the logic bellow: When rename, if the name without suffix then add it ,else use the origin name * simply server module configs (#1424) * move updateTaskState into try/catch block in case of exception * fix NPE * using conf.getInt instead of getString * for AbstractZKClient, remove the log, for it will print the same log message in createZNodePath. for AlertDao, correct the spelling. * duplicate * refactor getTaskWorkerGroupId * add friendly log * update hearbeat thread num = 1 * fix the bug when worker execute task using queue. and remove checking Tenant user anymore in TaskScheduleThread * 1. move verifyTaskInstanceIsNull after taskInstance 2. keep verifyTenantIsNull/verifyTaskInstanceIsNull clean and readable * fix the message * delete before check to avoid KeeperException$NoNodeException * fix the message * check processInstance state before delete tenant * check processInstance state before delete worker group * refactor * merge api constants into common constatns * update the resource perm * update the dataSource perm * fix CheckUtils.checkUserParams method * update AlertGroupService, extends from BaseService, remove duplicate methods * refactor * modify method name * add hasProjectAndPerm method * using checkProject instead of getResultStatus * delete checkAuth method, using hasProjectAndPerm instead. * correct spelling * add transactional for deleteWorkerGroupById * add Transactional for deleteProcessInstanceById method * change sqlSessionTemplate singleton * change sqlSessionTemplate singleton and reformat code * fix unsuitable error message * update shutdownhook methods * fix worker log bug * fix api server debug mode bug * upgrade zk version * delete this line ,for zkClient.close() will do the whole thing * fix master server shutdown error * degrade zk version and add FourLetterWordMain class * fix PathChildrenCache not close * add Transactional for createSession method * add more message for java-doc * delete App, let spring manage connectionFactory * add license * add class Application for test support * refactor masterServer and workerServer * add args * fix the spring transaction not work bug * remove author * delete @Bean annotation * delete master/worker properties * updates * rename application.properties to application-dao.properties * delete this class * delete master/worker properties and refactory master/worker * delete unused imports * merge * delete unused config
5 years ago
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.utils.AlertManager;
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
import org.apache.dolphinscheduler.service.process.ProcessService;
import org.apache.dolphinscheduler.service.quartz.cron.CronUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import static org.apache.dolphinscheduler.common.Constants.*;
/**
* master exec thread,split dag
*/
public class MasterExecThread implements Runnable {
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* logger of MasterExecThread
*/
private static final Logger logger = LoggerFactory.getLogger(MasterExecThread.class);
/**
* process instance
*/
private ProcessInstance processInstance;
/**
* runing TaskNode
*/
private final Map<MasterBaseTaskExecThread,Future<Boolean>> activeTaskNode = new ConcurrentHashMap<>();
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* task exec service
*/
private final ExecutorService taskExecService;
/**
* submit failure nodes
*/
private boolean taskFailedSubmit = false;
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* recover node id list
*/
private List<TaskInstance> recoverNodeIdList = new ArrayList<>();
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* error task list
*/
private Map<String,TaskInstance> errorTaskList = new ConcurrentHashMap<>();
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* complete task list
*/
private Map<String, TaskInstance> completeTaskList = new ConcurrentHashMap<>();
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* ready to submit task list
*/
private Map<String, TaskInstance> readyToSubmitTaskList = new ConcurrentHashMap<>();
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* depend failed task map
*/
private Map<String, TaskInstance> dependFailedTask = new ConcurrentHashMap<>();
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* forbidden task map
*/
private Map<String, TaskNode> forbiddenTaskList = new ConcurrentHashMap<>();
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* skip task map
*/
private Map<String, TaskNode> skipTaskNodeList = new ConcurrentHashMap<>();
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* recover tolerance fault task list
*/
private List<TaskInstance> recoverToleranceFaultTaskList = new ArrayList<>();
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* alert manager
*/
private AlertManager alertManager = new AlertManager();
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* the object of DAG
*/
private DAG<String,TaskNode,TaskNodeRelation> dag;
/**
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
* process service
*/
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
private ProcessService processService;
/**
merge dev-db to dev (#1426) * [dolphinscheduler-1345] [newfeature] Add DB2 Datasource (#1391) * Fix the problem that the 'queueId' is not present when creating a tenant based on the default queue. (#1409) * dolphinscheduler-#1403][bug]improve the check rules (#1408) 1. When check failed, we don’t know whitch parameter has is wrong, Because username、password、email and phone were checks together. I refactored the check method ,Now it will return failed msg by each field. 2. The email check regex support [_|\-|\.]?) in createUser.vue, But it do not support in backend server , I fix it, Now they have the same check regex both in frontend and backend * jcip-annotations define version information and maven-assembly-plugin add groupId (#1413) * "v-for" add key (#1419) * [dolphinscheduler-#1397] [bug]Resources can not be previewed or updated  (#1406) When create an resource the name will add the suffix, But When rename the resource there is no suffix add to the name, So When update resource name without suffix just like "test.sh" => "test" , Then the bug reproduced. To fix this bug i add the logic bellow: When rename, if the name without suffix then add it ,else use the origin name * simply server module configs (#1424) * move updateTaskState into try/catch block in case of exception * fix NPE * using conf.getInt instead of getString * for AbstractZKClient, remove the log, for it will print the same log message in createZNodePath. for AlertDao, correct the spelling. * duplicate * refactor getTaskWorkerGroupId * add friendly log * update hearbeat thread num = 1 * fix the bug when worker execute task using queue. and remove checking Tenant user anymore in TaskScheduleThread * 1. move verifyTaskInstanceIsNull after taskInstance 2. keep verifyTenantIsNull/verifyTaskInstanceIsNull clean and readable * fix the message * delete before check to avoid KeeperException$NoNodeException * fix the message * check processInstance state before delete tenant * check processInstance state before delete worker group * refactor * merge api constants into common constatns * update the resource perm * update the dataSource perm * fix CheckUtils.checkUserParams method * update AlertGroupService, extends from BaseService, remove duplicate methods * refactor * modify method name * add hasProjectAndPerm method * using checkProject instead of getResultStatus * delete checkAuth method, using hasProjectAndPerm instead. * correct spelling * add transactional for deleteWorkerGroupById * add Transactional for deleteProcessInstanceById method * change sqlSessionTemplate singleton * change sqlSessionTemplate singleton and reformat code * fix unsuitable error message * update shutdownhook methods * fix worker log bug * fix api server debug mode bug * upgrade zk version * delete this line ,for zkClient.close() will do the whole thing * fix master server shutdown error * degrade zk version and add FourLetterWordMain class * fix PathChildrenCache not close * add Transactional for createSession method * add more message for java-doc * delete App, let spring manage connectionFactory * add license * add class Application for test support * refactor masterServer and workerServer * add args * fix the spring transaction not work bug * remove author * delete @Bean annotation * delete master/worker properties * updates * rename application.properties to application-dao.properties * delete this class * delete master/worker properties and refactory master/worker * delete unused imports * merge * delete unused config
5 years ago
* master config
*/
merge dev-db to dev (#1426) * [dolphinscheduler-1345] [newfeature] Add DB2 Datasource (#1391) * Fix the problem that the 'queueId' is not present when creating a tenant based on the default queue. (#1409) * dolphinscheduler-#1403][bug]improve the check rules (#1408) 1. When check failed, we don’t know whitch parameter has is wrong, Because username、password、email and phone were checks together. I refactored the check method ,Now it will return failed msg by each field. 2. The email check regex support [_|\-|\.]?) in createUser.vue, But it do not support in backend server , I fix it, Now they have the same check regex both in frontend and backend * jcip-annotations define version information and maven-assembly-plugin add groupId (#1413) * "v-for" add key (#1419) * [dolphinscheduler-#1397] [bug]Resources can not be previewed or updated  (#1406) When create an resource the name will add the suffix, But When rename the resource there is no suffix add to the name, So When update resource name without suffix just like "test.sh" => "test" , Then the bug reproduced. To fix this bug i add the logic bellow: When rename, if the name without suffix then add it ,else use the origin name * simply server module configs (#1424) * move updateTaskState into try/catch block in case of exception * fix NPE * using conf.getInt instead of getString * for AbstractZKClient, remove the log, for it will print the same log message in createZNodePath. for AlertDao, correct the spelling. * duplicate * refactor getTaskWorkerGroupId * add friendly log * update hearbeat thread num = 1 * fix the bug when worker execute task using queue. and remove checking Tenant user anymore in TaskScheduleThread * 1. move verifyTaskInstanceIsNull after taskInstance 2. keep verifyTenantIsNull/verifyTaskInstanceIsNull clean and readable * fix the message * delete before check to avoid KeeperException$NoNodeException * fix the message * check processInstance state before delete tenant * check processInstance state before delete worker group * refactor * merge api constants into common constatns * update the resource perm * update the dataSource perm * fix CheckUtils.checkUserParams method * update AlertGroupService, extends from BaseService, remove duplicate methods * refactor * modify method name * add hasProjectAndPerm method * using checkProject instead of getResultStatus * delete checkAuth method, using hasProjectAndPerm instead. * correct spelling * add transactional for deleteWorkerGroupById * add Transactional for deleteProcessInstanceById method * change sqlSessionTemplate singleton * change sqlSessionTemplate singleton and reformat code * fix unsuitable error message * update shutdownhook methods * fix worker log bug * fix api server debug mode bug * upgrade zk version * delete this line ,for zkClient.close() will do the whole thing * fix master server shutdown error * degrade zk version and add FourLetterWordMain class * fix PathChildrenCache not close * add Transactional for createSession method * add more message for java-doc * delete App, let spring manage connectionFactory * add license * add class Application for test support * refactor masterServer and workerServer * add args * fix the spring transaction not work bug * remove author * delete @Bean annotation * delete master/worker properties * updates * rename application.properties to application-dao.properties * delete this class * delete master/worker properties and refactory master/worker * delete unused imports * merge * delete unused config
5 years ago
private MasterConfig masterConfig;
/**
*
*/
private NettyRemotingClient nettyRemotingClient;
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* constructor of MasterExecThread
4 years ago
* @param processInstance processInstance
* @param processService processService
* @param nettyRemotingClient nettyRemotingClient
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
*/
public MasterExecThread(ProcessInstance processInstance, ProcessService processService, NettyRemotingClient nettyRemotingClient){
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
this.processService = processService;
this.processInstance = processInstance;
merge dev-db to dev (#1426) * [dolphinscheduler-1345] [newfeature] Add DB2 Datasource (#1391) * Fix the problem that the 'queueId' is not present when creating a tenant based on the default queue. (#1409) * dolphinscheduler-#1403][bug]improve the check rules (#1408) 1. When check failed, we don’t know whitch parameter has is wrong, Because username、password、email and phone were checks together. I refactored the check method ,Now it will return failed msg by each field. 2. The email check regex support [_|\-|\.]?) in createUser.vue, But it do not support in backend server , I fix it, Now they have the same check regex both in frontend and backend * jcip-annotations define version information and maven-assembly-plugin add groupId (#1413) * "v-for" add key (#1419) * [dolphinscheduler-#1397] [bug]Resources can not be previewed or updated  (#1406) When create an resource the name will add the suffix, But When rename the resource there is no suffix add to the name, So When update resource name without suffix just like "test.sh" => "test" , Then the bug reproduced. To fix this bug i add the logic bellow: When rename, if the name without suffix then add it ,else use the origin name * simply server module configs (#1424) * move updateTaskState into try/catch block in case of exception * fix NPE * using conf.getInt instead of getString * for AbstractZKClient, remove the log, for it will print the same log message in createZNodePath. for AlertDao, correct the spelling. * duplicate * refactor getTaskWorkerGroupId * add friendly log * update hearbeat thread num = 1 * fix the bug when worker execute task using queue. and remove checking Tenant user anymore in TaskScheduleThread * 1. move verifyTaskInstanceIsNull after taskInstance 2. keep verifyTenantIsNull/verifyTaskInstanceIsNull clean and readable * fix the message * delete before check to avoid KeeperException$NoNodeException * fix the message * check processInstance state before delete tenant * check processInstance state before delete worker group * refactor * merge api constants into common constatns * update the resource perm * update the dataSource perm * fix CheckUtils.checkUserParams method * update AlertGroupService, extends from BaseService, remove duplicate methods * refactor * modify method name * add hasProjectAndPerm method * using checkProject instead of getResultStatus * delete checkAuth method, using hasProjectAndPerm instead. * correct spelling * add transactional for deleteWorkerGroupById * add Transactional for deleteProcessInstanceById method * change sqlSessionTemplate singleton * change sqlSessionTemplate singleton and reformat code * fix unsuitable error message * update shutdownhook methods * fix worker log bug * fix api server debug mode bug * upgrade zk version * delete this line ,for zkClient.close() will do the whole thing * fix master server shutdown error * degrade zk version and add FourLetterWordMain class * fix PathChildrenCache not close * add Transactional for createSession method * add more message for java-doc * delete App, let spring manage connectionFactory * add license * add class Application for test support * refactor masterServer and workerServer * add args * fix the spring transaction not work bug * remove author * delete @Bean annotation * delete master/worker properties * updates * rename application.properties to application-dao.properties * delete this class * delete master/worker properties and refactory master/worker * delete unused imports * merge * delete unused config
5 years ago
this.masterConfig = SpringApplicationContext.getBean(MasterConfig.class);
int masterTaskExecNum = masterConfig.getMasterExecTaskNum();
this.taskExecService = ThreadUtils.newDaemonFixedThreadExecutor("Master-Task-Exec-Thread",
masterTaskExecNum);
this.nettyRemotingClient = nettyRemotingClient;
}
@Override
public void run() {
// process instance is null
if (processInstance == null){
logger.info("process instance is not exists");
return;
}
// check to see if it's done
if (processInstance.getState().typeIsFinished()){
logger.info("process instance is done : {}",processInstance.getId());
return;
}
try {
if (processInstance.isComplementData() && Flag.NO == processInstance.getIsSubProcess()){
// sub process complement data
executeComplementProcess();
}else{
// execute flow
executeProcess();
}
}catch (Exception e){
logger.error("master exec thread exception", e);
logger.error("process execute failed, process id:{}", processInstance.getId());
processInstance.setState(ExecutionStatus.FAILURE);
processInstance.setEndTime(new Date());
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
processService.updateProcessInstance(processInstance);
}finally {
taskExecService.shutdown();
// post handle
postHandle();
}
}
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* execute process
* @throws Exception exception
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
*/
private void executeProcess() throws Exception {
prepareProcess();
runProcess();
endProcess();
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* execute complement process
* @throws Exception exception
*/
private void executeComplementProcess() throws Exception {
Map<String, String> cmdParam = JSONUtils.toMap(processInstance.getCommandParam());
Date startDate = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_START_DATE));
Date endDate = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE));
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
processService.saveProcessInstance(processInstance);
// get schedules
int processDefinitionId = processInstance.getProcessDefinitionId();
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
List<Schedule> schedules = processService.queryReleaseSchedulerListByProcessDefinitionId(processDefinitionId);
List<Date> listDate = Lists.newLinkedList();
if(!CollectionUtils.isEmpty(schedules)){
for (Schedule schedule : schedules) {
listDate.addAll(CronUtils.getSelfFireDateList(startDate, endDate, schedule.getCrontab()));
}
}
// get first fire date
Iterator<Date> iterator = null;
Date scheduleDate = null;
if(!CollectionUtils.isEmpty(listDate)) {
iterator = listDate.iterator();
scheduleDate = iterator.next();
processInstance.setScheduleTime(scheduleDate);
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
processService.updateProcessInstance(processInstance);
}else{
scheduleDate = processInstance.getScheduleTime();
if(scheduleDate == null){
scheduleDate = startDate;
}
}
while(Stopper.isRunning()){
// prepare dag and other info
prepareProcess();
if(dag == null){
logger.error("process {} dag is null, please check out parameters",
processInstance.getId());
processInstance.setState(ExecutionStatus.SUCCESS);
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
processService.updateProcessInstance(processInstance);
return;
}
// execute process ,waiting for end
runProcess();
// process instance failure ,no more complements
if(!processInstance.getState().typeIsSuccess()){
logger.info("process {} state {}, complement not completely!",
processInstance.getId(), processInstance.getState());
break;
}
// current process instance success ,next execute
if(null == iterator){
// loop by day
scheduleDate = DateUtils.getSomeDay(scheduleDate, 1);
if(scheduleDate.after(endDate)){
// all success
logger.info("process {} complement completely!", processInstance.getId());
break;
}
}else{
// loop by schedule date
if(!iterator.hasNext()){
// all success
logger.info("process {} complement completely!", processInstance.getId());
break;
}
scheduleDate = iterator.next();
}
logger.info("process {} start to complement {} data",
processInstance.getId(), DateUtils.dateToString(scheduleDate));
// execute next process instance complement data
processInstance.setScheduleTime(scheduleDate);
if(cmdParam.containsKey(Constants.CMDPARAM_RECOVERY_START_NODE_STRING)){
cmdParam.remove(Constants.CMDPARAM_RECOVERY_START_NODE_STRING);
processInstance.setCommandParam(JSONUtils.toJson(cmdParam));
}
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
List<TaskInstance> taskInstanceList = processService.findValidTaskListByProcessId(processInstance.getId());
for(TaskInstance taskInstance : taskInstanceList){
taskInstance.setFlag(Flag.NO);
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
processService.updateTaskInstance(taskInstance);
}
processInstance.setState(ExecutionStatus.RUNNING_EXEUTION);
processInstance.setGlobalParams(ParameterUtils.curingGlobalParams(
processInstance.getProcessDefinition().getGlobalParamMap(),
processInstance.getProcessDefinition().getGlobalParamList(),
CommandType.COMPLEMENT_DATA, processInstance.getScheduleTime()));
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
processService.saveProcessInstance(processInstance);
}
// flow end
endProcess();
}
/**
* prepare process parameter
* @throws Exception exception
*/
private void prepareProcess() throws Exception {
// init task queue
initTaskQueue();
// gen process dag
buildFlowDag();
logger.info("prepare process :{} end", processInstance.getId());
}
/**
* process end handle
*/
private void endProcess() {
processInstance.setEndTime(new Date());
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
processService.updateProcessInstance(processInstance);
if(processInstance.getState().typeIsWaittingThread()){
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
processService.createRecoveryWaitingThreadCommand(null, processInstance);
}
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
List<TaskInstance> taskInstances = processService.findValidTaskListByProcessId(processInstance.getId());
alertManager.sendAlertProcessInstance(processInstance, taskInstances);
}
/**
* generate process dag
* @throws Exception exception
*/
private void buildFlowDag() throws Exception {
recoverNodeIdList = getStartTaskInstanceList(processInstance.getCommandParam());
forbiddenTaskList = DagHelper.getForbiddenTaskNodeMaps(processInstance.getProcessInstanceJson());
// generate process to get DAG info
List<String> recoveryNameList = getRecoveryNodeNameList();
List<String> startNodeNameList = parseStartNodeName(processInstance.getCommandParam());
ProcessDag processDag = generateFlowDag(processInstance.getProcessInstanceJson(),
startNodeNameList, recoveryNameList, processInstance.getTaskDependType());
if(processDag == null){
logger.error("processDag is null");
return;
}
// generate process dag
dag = DagHelper.buildDagGraph(processDag);
}
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* init task queue
*/
private void initTaskQueue(){
taskFailedSubmit = false;
activeTaskNode.clear();
dependFailedTask.clear();
completeTaskList.clear();
errorTaskList.clear();
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
List<TaskInstance> taskInstanceList = processService.findValidTaskListByProcessId(processInstance.getId());
for(TaskInstance task : taskInstanceList){
if(task.isTaskComplete()){
completeTaskList.put(task.getName(), task);
}
if(task.getState().typeIsFailure() && !task.taskCanRetry()){
errorTaskList.put(task.getName(), task);
}
}
}
/**
* process post handle
*/
private void postHandle() {
logger.info("develop mode is: {}", CommonUtils.isDevelopMode());
if (!CommonUtils.isDevelopMode()) {
// get exec dir
String execLocalPath = org.apache.dolphinscheduler.common.utils.FileUtils
.getProcessExecDir(processInstance.getProcessDefinition().getProjectId(),
processInstance.getProcessDefinitionId(),
processInstance.getId());
try {
FileUtils.deleteDirectory(new File(execLocalPath));
} catch (IOException e) {
logger.error("delete exec dir failed ", e);
}
}
}
/**
1,ZookeeperRegister use common.properties zookeeperRoot path 2,api start exclude org.apache.dolphinscheduler.server.* (#2307) * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * add license * modify javadoc error * TaskExecutionContext create modify * buildAckCommand taskInstanceId not set modify * java doc error modify * add comment * ExecutorManager interface add generic type * add TaskInstanceCacheManager receive Worker report result * TaskInstance setExecutePath * add TaskInstanceCacheManager to receive Worker Task result report * TaskInstanceCacheManager add remove method * add license * add dispatcht task method * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * 1,worker TaskPros use TaskExecutionContext replase 2,Master kill Task , KillTaskProcessor modify * worker remove db * ShellTask modify * master persistence processId and appIds * master persistence processId and appIds * master add kill task logic * master add kill task logic * master add kill task logic * javadoc error modify * remove chinese log * executeDirectly method add Override * remote module modify * TaskKillResponseProcessor command type modify * create buildKillCommand * host add host:port format * host add host:port format * TaskAckProcessor modify * TaskAckProcessor modify * task prioriry refator * remove ITaskQueue * task prioriry refator * remove ITaskQueue * TaskPriority refactor * remove logs * WorkerServer refactor * MasterSchedulerService modify * WorkerConfig listen port modify * modify master and worker listen port * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * Encapsulate the parameters required by sqltask * 1,Encapsulate the parameters required by sqltask 2,SQLTask optimization * AbstractTask modify * ProcedureTask optimization * MasterSchedulerService modify * TaskUpdateQueueConsumer modify * test * DataxTask process run debug * DataxTask process run debug * add protobuf dependency,MR、Spark task etc need this * TaskUpdateQueueConsumer modify * TaskExecutionContextBuilder set TaskInstance workgroup * WorkerGroupService queryAllGroup modify query available work group * 1,get workergroup from zk modify 2,SpringConnectionFactory repeat load modify * master and worker register ip use OSUtils.getHost() * ProcessInstance host set ip:port format * worker fault tolerance modify * Constants and .env modify * master fault tolerant bug modify * UT add pom.xml * timing online modify * when taskResponse is faster than taskAck to db,task state will error add async queue and new a thread reslove this problem * TaskExecutionContext set host * 1,TaskManager refactor 2, api start load server dolphinschedule-daemon.sh modify * 1,TaskManager refactor 2, api start load server dolphinschedule-daemon.sh modify * add UT in pom.xml * revert dolphinscheduler-daemon.sh * ZookeeperRegister use common.properties zookeeperRoot path * api start exclude org.apache.dolphinscheduler.server.* * ZookeeperRegister use common.properties zookeeperRoot path * 1,api start load server filter 2,SHELL task exitStatusCode modify * java doc error modify * java doc error modify * remove todo Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn>
4 years ago
* submit task to execute
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @param taskInstance task instance
* @return TaskInstance
*/
private TaskInstance submitTaskExec(TaskInstance taskInstance) {
MasterBaseTaskExecThread abstractExecThread = null;
if(taskInstance.isSubProcess()){
abstractExecThread = new SubProcessTaskExecThread(taskInstance);
}else if(taskInstance.isDependTask()){
abstractExecThread = new DependentTaskExecThread(taskInstance);
}else if(taskInstance.isConditionsTask()){
abstractExecThread = new ConditionsTaskExecThread(taskInstance);
}else {
abstractExecThread = new MasterTaskExecThread(taskInstance);
}
Future<Boolean> future = taskExecService.submit(abstractExecThread);
activeTaskNode.putIfAbsent(abstractExecThread, future);
return abstractExecThread.getTaskInstance();
}
/**
* find task instance in db.
* in case submit more than one same name task in the same time.
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @param taskName task name
* @return TaskInstance
*/
private TaskInstance findTaskIfExists(String taskName){
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
List<TaskInstance> taskInstanceList = processService.findValidTaskListByProcessId(this.processInstance.getId());
for(TaskInstance taskInstance : taskInstanceList){
if(taskInstance.getName().equals(taskName)){
return taskInstance;
}
}
return null;
}
/**
* encapsulation task
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @param processInstance process instance
* @param nodeName node name
* @return TaskInstance
*/
private TaskInstance createTaskInstance(ProcessInstance processInstance, String nodeName,
TaskNode taskNode) {
TaskInstance taskInstance = findTaskIfExists(nodeName);
if(taskInstance == null){
taskInstance = new TaskInstance();
// task name
taskInstance.setName(nodeName);
// process instance define id
taskInstance.setProcessDefinitionId(processInstance.getProcessDefinitionId());
// task instance state
taskInstance.setState(ExecutionStatus.SUBMITTED_SUCCESS);
// process instance id
taskInstance.setProcessInstanceId(processInstance.getId());
// task instance node json
taskInstance.setTaskJson(JSON.toJSONString(taskNode));
// task instance type
taskInstance.setTaskType(taskNode.getType());
// task instance whether alert
taskInstance.setAlertFlag(Flag.NO);
// task instance start time
taskInstance.setStartTime(new Date());
// task instance flag
taskInstance.setFlag(Flag.YES);
// task instance retry times
taskInstance.setRetryTimes(0);
// max task instance retry times
taskInstance.setMaxRetryTimes(taskNode.getMaxRetryTimes());
// retry task instance interval
taskInstance.setRetryInterval(taskNode.getRetryInterval());
// task instance priority
if(taskNode.getTaskInstancePriority() == null){
taskInstance.setTaskInstancePriority(Priority.MEDIUM);
}else{
taskInstance.setTaskInstancePriority(taskNode.getTaskInstancePriority());
}
String processWorkerGroup = processInstance.getWorkerGroup();
processWorkerGroup = StringUtils.isBlank(processWorkerGroup) ? DEFAULT_WORKER_GROUP : processWorkerGroup;
String taskWorkerGroup = StringUtils.isBlank(taskNode.getWorkerGroup()) ? processWorkerGroup : taskNode.getWorkerGroup();
if (!processWorkerGroup.equals(DEFAULT_WORKER_GROUP) && taskWorkerGroup.equals(DEFAULT_WORKER_GROUP)) {
taskInstance.setWorkerGroup(processWorkerGroup);
}else {
taskInstance.setWorkerGroup(taskWorkerGroup);
}
}
return taskInstance;
}
/**
* if all of the task dependence are skip, skip it too.
* @param taskNode
* @return
*/
private boolean isTaskNodeNeedSkip(TaskNode taskNode){
if(CollectionUtils.isEmpty(taskNode.getDepList())){
return false;
}
for(String depNode : taskNode.getDepList()){
if(!skipTaskNodeList.containsKey(depNode)){
return false;
}
}
return true;
}
/**
* set task node skip if dependence all skip
* @param taskNodesSkipList
*/
private void setTaskNodeSkip(List<String> taskNodesSkipList){
for(String skipNode : taskNodesSkipList){
skipTaskNodeList.putIfAbsent(skipNode, dag.getNode(skipNode));
Collection<String> postNodeList = DagHelper.getStartVertex(skipNode, dag, completeTaskList);
List<String> postSkipList = new ArrayList<>();
for(String post : postNodeList){
TaskNode postNode = dag.getNode(post);
if(isTaskNodeNeedSkip(postNode)){
postSkipList.add(post);
}
}
setTaskNodeSkip(postSkipList);
}
}
/**
* parse condition task find the branch process
* set skip flag for another one.
* @param nodeName
* @return
*/
private List<String> parseConditionTask(String nodeName){
List<String> conditionTaskList = new ArrayList<>();
TaskNode taskNode = dag.getNode(nodeName);
if(!taskNode.isConditionsTask()){
return conditionTaskList;
}
ConditionsParameters conditionsParameters =
JSONUtils.parseObject(taskNode.getConditionResult(), ConditionsParameters.class);
TaskInstance taskInstance = completeTaskList.get(nodeName);
if(taskInstance == null){
logger.error("task instance {} cannot find, please check it!", nodeName);
return conditionTaskList;
}
if(taskInstance.getState().typeIsSuccess()){
conditionTaskList = conditionsParameters.getSuccessNode();
setTaskNodeSkip(conditionsParameters.getFailedNode());
}else if(taskInstance.getState().typeIsFailure()){
conditionTaskList = conditionsParameters.getFailedNode();
setTaskNodeSkip(conditionsParameters.getSuccessNode());
}else{
conditionTaskList.add(nodeName);
}
return conditionTaskList;
}
/**
* parse post node list of previous node
* if condition node: return process according to the settings
* if post node completed, return post nodes of the completed node
* @param previousNodeName
* @return
*/
private List<String> parsePostNodeList(String previousNodeName){
List<String> postNodeList = new ArrayList<>();
TaskNode taskNode = dag.getNode(previousNodeName);
if(taskNode != null && taskNode.isConditionsTask()){
return parseConditionTask(previousNodeName);
}
Collection<String> postNodeCollection = DagHelper.getStartVertex(previousNodeName, dag, completeTaskList);
List<String> postSkipList = new ArrayList<>();
// delete success node, parse the past nodes
// if conditions node,
// 1. parse the branch process according the conditions setting
// 2. set skip flag on anther branch process
for(String postNode : postNodeCollection){
if(completeTaskList.containsKey(postNode)){
TaskInstance postTaskInstance = completeTaskList.get(postNode);
if(dag.getNode(postNode).isConditionsTask()){
List<String> conditionTaskNodeList = parseConditionTask(postNode);
for(String conditions : conditionTaskNodeList){
postNodeList.addAll(parsePostNodeList(conditions));
}
}else if(postTaskInstance.getState().typeIsSuccess()){
postNodeList.addAll(parsePostNodeList(postNode));
}else{
postNodeList.add(postNode);
}
}else if(isTaskNodeNeedSkip(dag.getNode(postNode))){
postSkipList.add(postNode);
setTaskNodeSkip(postSkipList);
postSkipList.clear();
}else{
postNodeList.add(postNode);
}
}
return postNodeList;
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* submit post node
* @param parentNodeName parent node name
*/
private void submitPostNode(String parentNodeName){
List<String> submitTaskNodeList = parsePostNodeList(parentNodeName);
List<TaskInstance> taskInstances = new ArrayList<>();
for(String taskNode : submitTaskNodeList){
taskInstances.add(createTaskInstance(processInstance, taskNode,
dag.getNode(taskNode)));
}
// if previous node success , post node submit
for(TaskInstance task : taskInstances){
if(readyToSubmitTaskList.containsKey(task.getName())){
continue;
}
if(completeTaskList.containsKey(task.getName())){
logger.info("task {} has already run success", task.getName());
continue;
}
if(task.getState().typeIsPause() || task.getState().typeIsCancel()){
logger.info("task {} stopped, the state is {}", task.getName(), task.getState());
}else{
addTaskToStandByList(task);
}
}
}
/**
* determine whether the dependencies of the task node are complete
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @return DependResult
*/
private DependResult isTaskDepsComplete(String taskName) {
Collection<String> startNodes = dag.getBeginNode();
// if vertex,returns true directly
if(startNodes.contains(taskName)){
return DependResult.SUCCESS;
}
TaskNode taskNode = dag.getNode(taskName);
List<String> depNameList = taskNode.getDepList();
for(String depsNode : depNameList ){
if(!dag.containsNode(depsNode)
|| forbiddenTaskList.containsKey(depsNode)
|| skipTaskNodeList.containsKey(depsNode)){
continue;
}
// dependencies must be fully completed
if(!completeTaskList.containsKey(depsNode)){
return DependResult.WAITING;
}
ExecutionStatus depTaskState = completeTaskList.get(depsNode).getState();
// conditions task would not return failed.
if(depTaskState.typeIsFailure()
&& !DagHelper.haveConditionsAfterNode(depsNode, dag )
&& !dag.getNode(depsNode).isConditionsTask()){
return DependResult.FAILED;
}
if(depTaskState.typeIsPause() || depTaskState.typeIsCancel()){
return DependResult.WAITING;
}
}
logger.info("taskName: {} completeDependTaskList: {}", taskName, Arrays.toString(completeTaskList.keySet().toArray()));
return DependResult.SUCCESS;
}
/**
* query task instance by complete state
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @param state state
* @return task instance list
*/
private List<TaskInstance> getCompleteTaskByState(ExecutionStatus state){
List<TaskInstance> resultList = new ArrayList<>();
for (Map.Entry<String, TaskInstance> entry: completeTaskList.entrySet()) {
if(entry.getValue().getState() == state){
resultList.add(entry.getValue());
}
}
return resultList;
}
/**
* where there are ongoing tasks
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @param state state
* @return ExecutionStatus
*/
private ExecutionStatus runningState(ExecutionStatus state){
if(state == ExecutionStatus.READY_STOP ||
state == ExecutionStatus.READY_PAUSE ||
state == ExecutionStatus.WAITTING_THREAD){
// if the running task is not completed, the state remains unchanged
return state;
}else{
return ExecutionStatus.RUNNING_EXEUTION;
}
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* exists failure task,contains submit failuredependency failure,execute failure(retry after)
*
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @return Boolean whether has failed task
*/
private boolean hasFailedTask(){
if(this.taskFailedSubmit){
return true;
}
if(this.errorTaskList.size() > 0){
return true;
}
return this.dependFailedTask.size() > 0;
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* process instance failure
*
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @return Boolean whether process instance failed
*/
private boolean processFailed(){
if(hasFailedTask()) {
if(processInstance.getFailureStrategy() == FailureStrategy.END){
return true;
}
if (processInstance.getFailureStrategy() == FailureStrategy.CONTINUE) {
return readyToSubmitTaskList.size() == 0 || activeTaskNode.size() == 0;
}
}
return false;
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* whether task for waiting thread
* @return Boolean whether has waiting thread task
*/
private boolean hasWaitingThreadTask(){
List<TaskInstance> waitingList = getCompleteTaskByState(ExecutionStatus.WAITTING_THREAD);
return CollectionUtils.isNotEmpty(waitingList);
}
/**
* prepare for pause
* 1failed retry task in the preparation queue , returns to failure directly
* 2exists pause taskcomplement not completed, pending submission of tasks, return to suspension
* 3success
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @return ExecutionStatus
*/
private ExecutionStatus processReadyPause(){
if(hasRetryTaskInStandBy()){
return ExecutionStatus.FAILURE;
}
List<TaskInstance> pauseList = getCompleteTaskByState(ExecutionStatus.PAUSE);
if(CollectionUtils.isNotEmpty(pauseList)
|| !isComplementEnd()
|| readyToSubmitTaskList.size() > 0){
return ExecutionStatus.PAUSE;
}else{
return ExecutionStatus.SUCCESS;
}
}
/**
* generate the latest process instance status by the tasks state
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @return process instance execution status
*/
private ExecutionStatus getProcessInstanceState(){
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
ProcessInstance instance = processService.findProcessInstanceById(processInstance.getId());
ExecutionStatus state = instance.getState();
UT Coverage rate test (#2276) * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * add license * modify javadoc error * TaskExecutionContext create modify * buildAckCommand taskInstanceId not set modify * java doc error modify * add comment * ExecutorManager interface add generic type * add TaskInstanceCacheManager receive Worker report result * TaskInstance setExecutePath * add TaskInstanceCacheManager to receive Worker Task result report * TaskInstanceCacheManager add remove method * add license * add dispatcht task method * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * 1,worker TaskPros use TaskExecutionContext replase 2,Master kill Task , KillTaskProcessor modify * worker remove db * ShellTask modify * master persistence processId and appIds * master persistence processId and appIds * master add kill task logic * master add kill task logic * master add kill task logic * javadoc error modify * remove chinese log * executeDirectly method add Override * remote module modify * TaskKillResponseProcessor command type modify * create buildKillCommand * host add host:port format * host add host:port format * TaskAckProcessor modify * TaskAckProcessor modify * task prioriry refator * remove ITaskQueue * task prioriry refator * remove ITaskQueue * TaskPriority refactor * remove logs * WorkerServer refactor * MasterSchedulerService modify * WorkerConfig listen port modify * modify master and worker listen port * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * Encapsulate the parameters required by sqltask * 1,Encapsulate the parameters required by sqltask 2,SQLTask optimization * AbstractTask modify * ProcedureTask optimization * MasterSchedulerService modify * TaskUpdateQueueConsumer modify * test * DataxTask process run debug * DataxTask process run debug * add protobuf dependency,MR、Spark task etc need this * TaskUpdateQueueConsumer modify * TaskExecutionContextBuilder set TaskInstance workgroup * WorkerGroupService queryAllGroup modify query available work group * 1,get workergroup from zk modify 2,SpringConnectionFactory repeat load modify * master and worker register ip use OSUtils.getHost() * ProcessInstance host set ip:port format * worker fault tolerance modify * Constants and .env modify * master fault tolerant bug modify * UT add pom.xml Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn>
4 years ago
if(activeTaskNode.size() > 0 || haveRetryTaskStandBy()){
return runningState(state);
}
// process failure
if(processFailed()){
return ExecutionStatus.FAILURE;
}
// waiting thread
if(hasWaitingThreadTask()){
return ExecutionStatus.WAITTING_THREAD;
}
// pause
if(state == ExecutionStatus.READY_PAUSE){
return processReadyPause();
}
// stop
if(state == ExecutionStatus.READY_STOP){
List<TaskInstance> stopList = getCompleteTaskByState(ExecutionStatus.STOP);
List<TaskInstance> killList = getCompleteTaskByState(ExecutionStatus.KILL);
if(CollectionUtils.isNotEmpty(stopList)
|| CollectionUtils.isNotEmpty(killList) || !isComplementEnd()){
return ExecutionStatus.STOP;
}else{
return ExecutionStatus.SUCCESS;
}
}
// success
if(state == ExecutionStatus.RUNNING_EXEUTION){
if(readyToSubmitTaskList.size() > 0){
//tasks currently pending submission, no retries, indicating that depend is waiting to complete
return ExecutionStatus.RUNNING_EXEUTION;
}else{
// if the waiting queue is empty and the status is in progress, then success
return ExecutionStatus.SUCCESS;
}
}
return state;
}
UT Coverage rate test (#2276) * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * add license * modify javadoc error * TaskExecutionContext create modify * buildAckCommand taskInstanceId not set modify * java doc error modify * add comment * ExecutorManager interface add generic type * add TaskInstanceCacheManager receive Worker report result * TaskInstance setExecutePath * add TaskInstanceCacheManager to receive Worker Task result report * TaskInstanceCacheManager add remove method * add license * add dispatcht task method * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * 1,worker TaskPros use TaskExecutionContext replase 2,Master kill Task , KillTaskProcessor modify * worker remove db * ShellTask modify * master persistence processId and appIds * master persistence processId and appIds * master add kill task logic * master add kill task logic * master add kill task logic * javadoc error modify * remove chinese log * executeDirectly method add Override * remote module modify * TaskKillResponseProcessor command type modify * create buildKillCommand * host add host:port format * host add host:port format * TaskAckProcessor modify * TaskAckProcessor modify * task prioriry refator * remove ITaskQueue * task prioriry refator * remove ITaskQueue * TaskPriority refactor * remove logs * WorkerServer refactor * MasterSchedulerService modify * WorkerConfig listen port modify * modify master and worker listen port * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * Encapsulate the parameters required by sqltask * 1,Encapsulate the parameters required by sqltask 2,SQLTask optimization * AbstractTask modify * ProcedureTask optimization * MasterSchedulerService modify * TaskUpdateQueueConsumer modify * test * DataxTask process run debug * DataxTask process run debug * add protobuf dependency,MR、Spark task etc need this * TaskUpdateQueueConsumer modify * TaskExecutionContextBuilder set TaskInstance workgroup * WorkerGroupService queryAllGroup modify query available work group * 1,get workergroup from zk modify 2,SpringConnectionFactory repeat load modify * master and worker register ip use OSUtils.getHost() * ProcessInstance host set ip:port format * worker fault tolerance modify * Constants and .env modify * master fault tolerant bug modify * UT add pom.xml Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn>
4 years ago
/**
* whether standby task list have retry tasks
* @return
*/
private boolean haveRetryTaskStandBy() {
boolean result = false;
for(String taskName : readyToSubmitTaskList.keySet()){
TaskInstance task = readyToSubmitTaskList.get(taskName);
if(task.getState().typeIsFailure()){
result = true;
break;
}
}
return result;
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* whether complement end
* @return Boolean whether is complement end
*/
private boolean isComplementEnd() {
if(!processInstance.isComplementData()){
return true;
}
try {
Map<String, String> cmdParam = JSONUtils.toMap(processInstance.getCommandParam());
Date endTime = DateUtils.getScheduleDate(cmdParam.get(CMDPARAM_COMPLEMENT_DATA_END_DATE));
return processInstance.getScheduleTime().equals(endTime);
} catch (Exception e) {
logger.error("complement end failed ",e);
return false;
}
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* updateProcessInstance process instance state
* after each batch of tasks is executed, the status of the process instance is updated
*/
private void updateProcessInstanceState() {
ExecutionStatus state = getProcessInstanceState();
if(processInstance.getState() != state){
logger.info(
"work flow process instance [id: {}, name:{}], state change from {} to {}, cmd type: {}",
processInstance.getId(), processInstance.getName(),
processInstance.getState(), state,
processInstance.getCommandType());
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
ProcessInstance instance = processService.findProcessInstanceById(processInstance.getId());
instance.setState(state);
instance.setProcessDefinition(processInstance.getProcessDefinition());
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
processService.updateProcessInstance(instance);
processInstance = instance;
}
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* get task dependency result
* @param taskInstance task instance
* @return DependResult
*/
private DependResult getDependResultForTask(TaskInstance taskInstance){
return isTaskDepsComplete(taskInstance.getName());
}
/**
* add task to standby list
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @param taskInstance task instance
*/
private void addTaskToStandByList(TaskInstance taskInstance){
logger.info("add task to stand by list: {}", taskInstance.getName());
readyToSubmitTaskList.putIfAbsent(taskInstance.getName(), taskInstance);
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* remove task from stand by list
* @param taskInstance task instance
*/
private void removeTaskFromStandbyList(TaskInstance taskInstance){
logger.info("remove task from stand by list: {}", taskInstance.getName());
readyToSubmitTaskList.remove(taskInstance.getName());
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* has retry task in standby
* @return Boolean whether has retry task in standby
*/
private boolean hasRetryTaskInStandBy(){
for (Map.Entry<String, TaskInstance> entry: readyToSubmitTaskList.entrySet()) {
if(entry.getValue().getState().typeIsFailure()){
return true;
}
}
return false;
}
/**
* submit and watch the tasks, until the work flow stop
*/
private void runProcess(){
// submit start node
submitPostNode(null);
boolean sendTimeWarning = false;
while(!processInstance.isProcessInstanceStop()){
// send warning email if process time out.
4 years ago
if(!sendTimeWarning && checkProcessTimeOut(processInstance) ){
alertManager.sendProcessTimeoutAlert(processInstance,
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
processService.findProcessDefineById(processInstance.getProcessDefinitionId()));
sendTimeWarning = true;
}
for(Map.Entry<MasterBaseTaskExecThread,Future<Boolean>> entry: activeTaskNode.entrySet()) {
Future<Boolean> future = entry.getValue();
TaskInstance task = entry.getKey().getTaskInstance();
if(!future.isDone()){
continue;
}
UT Coverage rate test (#2276) * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * add license * modify javadoc error * TaskExecutionContext create modify * buildAckCommand taskInstanceId not set modify * java doc error modify * add comment * ExecutorManager interface add generic type * add TaskInstanceCacheManager receive Worker report result * TaskInstance setExecutePath * add TaskInstanceCacheManager to receive Worker Task result report * TaskInstanceCacheManager add remove method * add license * add dispatcht task method * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * 1,worker TaskPros use TaskExecutionContext replase 2,Master kill Task , KillTaskProcessor modify * worker remove db * ShellTask modify * master persistence processId and appIds * master persistence processId and appIds * master add kill task logic * master add kill task logic * master add kill task logic * javadoc error modify * remove chinese log * executeDirectly method add Override * remote module modify * TaskKillResponseProcessor command type modify * create buildKillCommand * host add host:port format * host add host:port format * TaskAckProcessor modify * TaskAckProcessor modify * task prioriry refator * remove ITaskQueue * task prioriry refator * remove ITaskQueue * TaskPriority refactor * remove logs * WorkerServer refactor * MasterSchedulerService modify * WorkerConfig listen port modify * modify master and worker listen port * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * Encapsulate the parameters required by sqltask * 1,Encapsulate the parameters required by sqltask 2,SQLTask optimization * AbstractTask modify * ProcedureTask optimization * MasterSchedulerService modify * TaskUpdateQueueConsumer modify * test * DataxTask process run debug * DataxTask process run debug * add protobuf dependency,MR、Spark task etc need this * TaskUpdateQueueConsumer modify * TaskExecutionContextBuilder set TaskInstance workgroup * WorkerGroupService queryAllGroup modify query available work group * 1,get workergroup from zk modify 2,SpringConnectionFactory repeat load modify * master and worker register ip use OSUtils.getHost() * ProcessInstance host set ip:port format * worker fault tolerance modify * Constants and .env modify * master fault tolerant bug modify * UT add pom.xml Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn>
4 years ago
// node monitor thread complete
UT Coverage rate test (#2276) * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * add license * modify javadoc error * TaskExecutionContext create modify * buildAckCommand taskInstanceId not set modify * java doc error modify * add comment * ExecutorManager interface add generic type * add TaskInstanceCacheManager receive Worker report result * TaskInstance setExecutePath * add TaskInstanceCacheManager to receive Worker Task result report * TaskInstanceCacheManager add remove method * add license * add dispatcht task method * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * 1,worker TaskPros use TaskExecutionContext replase 2,Master kill Task , KillTaskProcessor modify * worker remove db * ShellTask modify * master persistence processId and appIds * master persistence processId and appIds * master add kill task logic * master add kill task logic * master add kill task logic * javadoc error modify * remove chinese log * executeDirectly method add Override * remote module modify * TaskKillResponseProcessor command type modify * create buildKillCommand * host add host:port format * host add host:port format * TaskAckProcessor modify * TaskAckProcessor modify * task prioriry refator * remove ITaskQueue * task prioriry refator * remove ITaskQueue * TaskPriority refactor * remove logs * WorkerServer refactor * MasterSchedulerService modify * WorkerConfig listen port modify * modify master and worker listen port * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * Encapsulate the parameters required by sqltask * 1,Encapsulate the parameters required by sqltask 2,SQLTask optimization * AbstractTask modify * ProcedureTask optimization * MasterSchedulerService modify * TaskUpdateQueueConsumer modify * test * DataxTask process run debug * DataxTask process run debug * add protobuf dependency,MR、Spark task etc need this * TaskUpdateQueueConsumer modify * TaskExecutionContextBuilder set TaskInstance workgroup * WorkerGroupService queryAllGroup modify query available work group * 1,get workergroup from zk modify 2,SpringConnectionFactory repeat load modify * master and worker register ip use OSUtils.getHost() * ProcessInstance host set ip:port format * worker fault tolerance modify * Constants and .env modify * master fault tolerant bug modify * UT add pom.xml Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn>
4 years ago
task = this.processService.findTaskInstanceById(task.getId());
if(task == null){
this.taskFailedSubmit = true;
UT Coverage rate test (#2276) * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * add license * modify javadoc error * TaskExecutionContext create modify * buildAckCommand taskInstanceId not set modify * java doc error modify * add comment * ExecutorManager interface add generic type * add TaskInstanceCacheManager receive Worker report result * TaskInstance setExecutePath * add TaskInstanceCacheManager to receive Worker Task result report * TaskInstanceCacheManager add remove method * add license * add dispatcht task method * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * 1,worker TaskPros use TaskExecutionContext replase 2,Master kill Task , KillTaskProcessor modify * worker remove db * ShellTask modify * master persistence processId and appIds * master persistence processId and appIds * master add kill task logic * master add kill task logic * master add kill task logic * javadoc error modify * remove chinese log * executeDirectly method add Override * remote module modify * TaskKillResponseProcessor command type modify * create buildKillCommand * host add host:port format * host add host:port format * TaskAckProcessor modify * TaskAckProcessor modify * task prioriry refator * remove ITaskQueue * task prioriry refator * remove ITaskQueue * TaskPriority refactor * remove logs * WorkerServer refactor * MasterSchedulerService modify * WorkerConfig listen port modify * modify master and worker listen port * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * Encapsulate the parameters required by sqltask * 1,Encapsulate the parameters required by sqltask 2,SQLTask optimization * AbstractTask modify * ProcedureTask optimization * MasterSchedulerService modify * TaskUpdateQueueConsumer modify * test * DataxTask process run debug * DataxTask process run debug * add protobuf dependency,MR、Spark task etc need this * TaskUpdateQueueConsumer modify * TaskExecutionContextBuilder set TaskInstance workgroup * WorkerGroupService queryAllGroup modify query available work group * 1,get workergroup from zk modify 2,SpringConnectionFactory repeat load modify * master and worker register ip use OSUtils.getHost() * ProcessInstance host set ip:port format * worker fault tolerance modify * Constants and .env modify * master fault tolerant bug modify * UT add pom.xml Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn>
4 years ago
activeTaskNode.remove(entry.getKey());
continue;
}
UT Coverage rate test (#2276) * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * add license * modify javadoc error * TaskExecutionContext create modify * buildAckCommand taskInstanceId not set modify * java doc error modify * add comment * ExecutorManager interface add generic type * add TaskInstanceCacheManager receive Worker report result * TaskInstance setExecutePath * add TaskInstanceCacheManager to receive Worker Task result report * TaskInstanceCacheManager add remove method * add license * add dispatcht task method * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * 1,worker TaskPros use TaskExecutionContext replase 2,Master kill Task , KillTaskProcessor modify * worker remove db * ShellTask modify * master persistence processId and appIds * master persistence processId and appIds * master add kill task logic * master add kill task logic * master add kill task logic * javadoc error modify * remove chinese log * executeDirectly method add Override * remote module modify * TaskKillResponseProcessor command type modify * create buildKillCommand * host add host:port format * host add host:port format * TaskAckProcessor modify * TaskAckProcessor modify * task prioriry refator * remove ITaskQueue * task prioriry refator * remove ITaskQueue * TaskPriority refactor * remove logs * WorkerServer refactor * MasterSchedulerService modify * WorkerConfig listen port modify * modify master and worker listen port * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * Encapsulate the parameters required by sqltask * 1,Encapsulate the parameters required by sqltask 2,SQLTask optimization * AbstractTask modify * ProcedureTask optimization * MasterSchedulerService modify * TaskUpdateQueueConsumer modify * test * DataxTask process run debug * DataxTask process run debug * add protobuf dependency,MR、Spark task etc need this * TaskUpdateQueueConsumer modify * TaskExecutionContextBuilder set TaskInstance workgroup * WorkerGroupService queryAllGroup modify query available work group * 1,get workergroup from zk modify 2,SpringConnectionFactory repeat load modify * master and worker register ip use OSUtils.getHost() * ProcessInstance host set ip:port format * worker fault tolerance modify * Constants and .env modify * master fault tolerant bug modify * UT add pom.xml Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn>
4 years ago
// node monitor thread complete
if(task.getState().typeIsFinished()){
activeTaskNode.remove(entry.getKey());
}
logger.info("task :{}, id:{} complete, state is {} ",
task.getName(), task.getId(), task.getState());
1,ZookeeperRegister use common.properties zookeeperRoot path 2,api start exclude org.apache.dolphinscheduler.server.* (#2307) * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * 1, master persistent task 2. extract master and worker communication model * add license * modify javadoc error * TaskExecutionContext create modify * buildAckCommand taskInstanceId not set modify * java doc error modify * add comment * ExecutorManager interface add generic type * add TaskInstanceCacheManager receive Worker report result * TaskInstance setExecutePath * add TaskInstanceCacheManager to receive Worker Task result report * TaskInstanceCacheManager add remove method * add license * add dispatcht task method * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * AbstractCommandExecutor remove db access * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * taskInstanceCache is null ,need load from db * 1,worker TaskPros use TaskExecutionContext replase 2,Master kill Task , KillTaskProcessor modify * worker remove db * ShellTask modify * master persistence processId and appIds * master persistence processId and appIds * master add kill task logic * master add kill task logic * master add kill task logic * javadoc error modify * remove chinese log * executeDirectly method add Override * remote module modify * TaskKillResponseProcessor command type modify * create buildKillCommand * host add host:port format * host add host:port format * TaskAckProcessor modify * TaskAckProcessor modify * task prioriry refator * remove ITaskQueue * task prioriry refator * remove ITaskQueue * TaskPriority refactor * remove logs * WorkerServer refactor * MasterSchedulerService modify * WorkerConfig listen port modify * modify master and worker listen port * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * cancelTaskInstance set TaskExecutionContext host,logPath,executePath * Encapsulate the parameters required by sqltask * 1,Encapsulate the parameters required by sqltask 2,SQLTask optimization * AbstractTask modify * ProcedureTask optimization * MasterSchedulerService modify * TaskUpdateQueueConsumer modify * test * DataxTask process run debug * DataxTask process run debug * add protobuf dependency,MR、Spark task etc need this * TaskUpdateQueueConsumer modify * TaskExecutionContextBuilder set TaskInstance workgroup * WorkerGroupService queryAllGroup modify query available work group * 1,get workergroup from zk modify 2,SpringConnectionFactory repeat load modify * master and worker register ip use OSUtils.getHost() * ProcessInstance host set ip:port format * worker fault tolerance modify * Constants and .env modify * master fault tolerant bug modify * UT add pom.xml * timing online modify * when taskResponse is faster than taskAck to db,task state will error add async queue and new a thread reslove this problem * TaskExecutionContext set host * 1,TaskManager refactor 2, api start load server dolphinschedule-daemon.sh modify * 1,TaskManager refactor 2, api start load server dolphinschedule-daemon.sh modify * add UT in pom.xml * revert dolphinscheduler-daemon.sh * ZookeeperRegister use common.properties zookeeperRoot path * api start exclude org.apache.dolphinscheduler.server.* * ZookeeperRegister use common.properties zookeeperRoot path * 1,api start load server filter 2,SHELL task exitStatusCode modify * java doc error modify * java doc error modify * remove todo Co-authored-by: qiaozhanwei <qiaozhanwei@analysys.com.cn>
4 years ago
// node success , post node submit
if(task.getState() == ExecutionStatus.SUCCESS){
completeTaskList.put(task.getName(), task);
submitPostNode(task.getName());
continue;
}
// node fails, retry first, and then execute the failure process
if(task.getState().typeIsFailure()){
if(task.getState() == ExecutionStatus.NEED_FAULT_TOLERANCE){
this.recoverToleranceFaultTaskList.add(task);
}
if(task.taskCanRetry()){
addTaskToStandByList(task);
}else{
completeTaskList.put(task.getName(), task);
if( task.isConditionsTask()
|| DagHelper.haveConditionsAfterNode(task.getName(), dag)) {
submitPostNode(task.getName());
}else{
errorTaskList.put(task.getName(), task);
if(processInstance.getFailureStrategy() == FailureStrategy.END){
killTheOtherTasks();
}
}
}
continue;
}
// other status stop/pause
completeTaskList.put(task.getName(), task);
}
// send alert
if(CollectionUtils.isNotEmpty(this.recoverToleranceFaultTaskList)){
alertManager.sendAlertWorkerToleranceFault(processInstance, recoverToleranceFaultTaskList);
this.recoverToleranceFaultTaskList.clear();
}
// updateProcessInstance completed task status
// failure priority is higher than pause
// if a task fails, other suspended tasks need to be reset kill
if(errorTaskList.size() > 0){
for(Map.Entry<String, TaskInstance> entry: completeTaskList.entrySet()) {
TaskInstance completeTask = entry.getValue();
if(completeTask.getState()== ExecutionStatus.PAUSE){
completeTask.setState(ExecutionStatus.KILL);
completeTaskList.put(entry.getKey(), completeTask);
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
processService.updateTaskInstance(completeTask);
}
}
}
if(canSubmitTaskToQueue()){
submitStandByTask();
}
try {
Thread.sleep(Constants.SLEEP_TIME_MILLIS);
} catch (InterruptedException e) {
logger.error(e.getMessage(),e);
}
updateProcessInstanceState();
}
logger.info("process:{} end, state :{}", processInstance.getId(), processInstance.getState());
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* whether check process time out
* @param processInstance task instance
* @return true if time out of process instance > running time of process instance
*/
private boolean checkProcessTimeOut(ProcessInstance processInstance) {
if(processInstance.getTimeout() == 0 ){
return false;
}
Date now = new Date();
long runningTime = DateUtils.diffMin(now, processInstance.getStartTime());
return runningTime > processInstance.getTimeout();
}
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* whether can submit task to queue
* @return boolean
*/
private boolean canSubmitTaskToQueue() {
merge dev-db to dev (#1426) * [dolphinscheduler-1345] [newfeature] Add DB2 Datasource (#1391) * Fix the problem that the 'queueId' is not present when creating a tenant based on the default queue. (#1409) * dolphinscheduler-#1403][bug]improve the check rules (#1408) 1. When check failed, we don’t know whitch parameter has is wrong, Because username、password、email and phone were checks together. I refactored the check method ,Now it will return failed msg by each field. 2. The email check regex support [_|\-|\.]?) in createUser.vue, But it do not support in backend server , I fix it, Now they have the same check regex both in frontend and backend * jcip-annotations define version information and maven-assembly-plugin add groupId (#1413) * "v-for" add key (#1419) * [dolphinscheduler-#1397] [bug]Resources can not be previewed or updated  (#1406) When create an resource the name will add the suffix, But When rename the resource there is no suffix add to the name, So When update resource name without suffix just like "test.sh" => "test" , Then the bug reproduced. To fix this bug i add the logic bellow: When rename, if the name without suffix then add it ,else use the origin name * simply server module configs (#1424) * move updateTaskState into try/catch block in case of exception * fix NPE * using conf.getInt instead of getString * for AbstractZKClient, remove the log, for it will print the same log message in createZNodePath. for AlertDao, correct the spelling. * duplicate * refactor getTaskWorkerGroupId * add friendly log * update hearbeat thread num = 1 * fix the bug when worker execute task using queue. and remove checking Tenant user anymore in TaskScheduleThread * 1. move verifyTaskInstanceIsNull after taskInstance 2. keep verifyTenantIsNull/verifyTaskInstanceIsNull clean and readable * fix the message * delete before check to avoid KeeperException$NoNodeException * fix the message * check processInstance state before delete tenant * check processInstance state before delete worker group * refactor * merge api constants into common constatns * update the resource perm * update the dataSource perm * fix CheckUtils.checkUserParams method * update AlertGroupService, extends from BaseService, remove duplicate methods * refactor * modify method name * add hasProjectAndPerm method * using checkProject instead of getResultStatus * delete checkAuth method, using hasProjectAndPerm instead. * correct spelling * add transactional for deleteWorkerGroupById * add Transactional for deleteProcessInstanceById method * change sqlSessionTemplate singleton * change sqlSessionTemplate singleton and reformat code * fix unsuitable error message * update shutdownhook methods * fix worker log bug * fix api server debug mode bug * upgrade zk version * delete this line ,for zkClient.close() will do the whole thing * fix master server shutdown error * degrade zk version and add FourLetterWordMain class * fix PathChildrenCache not close * add Transactional for createSession method * add more message for java-doc * delete App, let spring manage connectionFactory * add license * add class Application for test support * refactor masterServer and workerServer * add args * fix the spring transaction not work bug * remove author * delete @Bean annotation * delete master/worker properties * updates * rename application.properties to application-dao.properties * delete this class * delete master/worker properties and refactory master/worker * delete unused imports * merge * delete unused config
5 years ago
return OSUtils.checkResource(masterConfig.getMasterMaxCpuloadAvg(), masterConfig.getMasterReservedMemory());
}
/**
add check user and definitions function when delete tenant (#1080) * update english documents * refactor zk client * update documents * update zkclient * update zkclient * update documents * add architecture-design * change i18n * update i18n * update english documents * add architecture-design * update english documents * update en-US documents * add architecture-design * update demo site * add mybatis plus model * modify mybatisplus * modify mybatisplus * change interface by mybatisplus * add unit test * refactor dao interface. * add unit test for dao... * add unit test for dao... * add unit test for dao... * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update some dao bugs * update for some bugs * update some bugs * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update * update * add multiply settings for application.yml * add multiply settings for application.yml * revert * update configuration settings in task record dao... * change application_master to application-master * change application_master to application-master * update application.yml to application.properties * revert * revert * add properties * add properties * revert * revert * add api start up.. add alert send try catch * update dao info level * fix bug: task cannot submit when recovery failover * fix bug: task cannot submit when recovery failover * merge from dev-db * revert * revert * fix bug: get process definition list failed. * fix bug: process instance interval is error * revert * revert * update * support stop submit success tasks * update kill process * update for stop process * update for stop process * add some logs for stop process * update for small bug. * add check strategy before submit task * revert * update * update * revert * wait task instance exists if null. * revert * update * change desc to description. * add check user and definitions function when delete tenant * update
5 years ago
* close the on going tasks
*/
add check user and definitions function when delete tenant (#1080) * update english documents * refactor zk client * update documents * update zkclient * update zkclient * update documents * add architecture-design * change i18n * update i18n * update english documents * add architecture-design * update english documents * update en-US documents * add architecture-design * update demo site * add mybatis plus model * modify mybatisplus * modify mybatisplus * change interface by mybatisplus * add unit test * refactor dao interface. * add unit test for dao... * add unit test for dao... * add unit test for dao... * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update some dao bugs * update for some bugs * update some bugs * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update * update * add multiply settings for application.yml * add multiply settings for application.yml * revert * update configuration settings in task record dao... * change application_master to application-master * change application_master to application-master * update application.yml to application.properties * revert * revert * add properties * add properties * revert * revert * add api start up.. add alert send try catch * update dao info level * fix bug: task cannot submit when recovery failover * fix bug: task cannot submit when recovery failover * merge from dev-db * revert * revert * fix bug: get process definition list failed. * fix bug: process instance interval is error * revert * revert * update * support stop submit success tasks * update kill process * update for stop process * update for stop process * add some logs for stop process * update for small bug. * add check strategy before submit task * revert * update * update * revert * wait task instance exists if null. * revert * update * change desc to description. * add check user and definitions function when delete tenant * update
5 years ago
private void killTheOtherTasks() {
logger.info("kill called on process instance id: {}, num: {}", processInstance.getId(),
activeTaskNode.size());
for (Map.Entry<MasterBaseTaskExecThread, Future<Boolean>> entry : activeTaskNode.entrySet()) {
MasterBaseTaskExecThread taskExecThread = entry.getKey();
Future<Boolean> future = entry.getValue();
add check user and definitions function when delete tenant (#1080) * update english documents * refactor zk client * update documents * update zkclient * update zkclient * update documents * add architecture-design * change i18n * update i18n * update english documents * add architecture-design * update english documents * update en-US documents * add architecture-design * update demo site * add mybatis plus model * modify mybatisplus * modify mybatisplus * change interface by mybatisplus * add unit test * refactor dao interface. * add unit test for dao... * add unit test for dao... * add unit test for dao... * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update some dao bugs * update for some bugs * update some bugs * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update * update * add multiply settings for application.yml * add multiply settings for application.yml * revert * update configuration settings in task record dao... * change application_master to application-master * change application_master to application-master * update application.yml to application.properties * revert * revert * add properties * add properties * revert * revert * add api start up.. add alert send try catch * update dao info level * fix bug: task cannot submit when recovery failover * fix bug: task cannot submit when recovery failover * merge from dev-db * revert * revert * fix bug: get process definition list failed. * fix bug: process instance interval is error * revert * revert * update * support stop submit success tasks * update kill process * update for stop process * update for stop process * add some logs for stop process * update for small bug. * add check strategy before submit task * revert * update * update * revert * wait task instance exists if null. * revert * update * change desc to description. * add check user and definitions function when delete tenant * update
5 years ago
TaskInstance taskInstance = taskExecThread.getTaskInstance();
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
taskInstance = processService.findTaskInstanceById(taskInstance.getId());
if(taskInstance != null && taskInstance.getState().typeIsFinished()){
add check user and definitions function when delete tenant (#1080) * update english documents * refactor zk client * update documents * update zkclient * update zkclient * update documents * add architecture-design * change i18n * update i18n * update english documents * add architecture-design * update english documents * update en-US documents * add architecture-design * update demo site * add mybatis plus model * modify mybatisplus * modify mybatisplus * change interface by mybatisplus * add unit test * refactor dao interface. * add unit test for dao... * add unit test for dao... * add unit test for dao... * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ScheduleMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProcessInstanceMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/ProjectUserMapper.xml # escheduler-dao/src/main/resources/cn.escheduler.dao.mapper/QueueMapper.xml # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProcessInstanceMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ProjectUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/QueueMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ResourceUserMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/ScheduleMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/SessionMapperTest.java # escheduler-dao/src/test/java/cn/escheduler/dao/mapper/TenantMapperTest.java * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update some dao bugs * update for some bugs * update some bugs * Merge remote-tracking branch 'upstream/dev-db' into dev-db # Conflicts: # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ProjectMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/ResourceMapper.xml # dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.xml * update * update * add multiply settings for application.yml * add multiply settings for application.yml * revert * update configuration settings in task record dao... * change application_master to application-master * change application_master to application-master * update application.yml to application.properties * revert * revert * add properties * add properties * revert * revert * add api start up.. add alert send try catch * update dao info level * fix bug: task cannot submit when recovery failover * fix bug: task cannot submit when recovery failover * merge from dev-db * revert * revert * fix bug: get process definition list failed. * fix bug: process instance interval is error * revert * revert * update * support stop submit success tasks * update kill process * update for stop process * update for stop process * add some logs for stop process * update for small bug. * add check strategy before submit task * revert * update * update * revert * wait task instance exists if null. * revert * update * change desc to description. * add check user and definitions function when delete tenant * update
5 years ago
continue;
}
if (!future.isDone()) {
// record kill info
logger.info("kill process instance, id: {}, task: {}", processInstance.getId(), taskExecThread.getTaskInstance().getId());
// kill node
taskExecThread.kill();
}
}
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* whether the retry interval is timed out
* @param taskInstance task instance
* @return Boolean
*/
private boolean retryTaskIntervalOverTime(TaskInstance taskInstance){
if(taskInstance.getState() != ExecutionStatus.FAILURE){
return true;
}
if(taskInstance.getId() == 0 ||
taskInstance.getMaxRetryTimes() ==0 ||
taskInstance.getRetryInterval() == 0 ){
return true;
}
Date now = new Date();
long failedTimeInterval = DateUtils.differSec(now, taskInstance.getEndTime());
// task retry does not over time, return false
return taskInstance.getRetryInterval() * SEC_2_MINUTES_TIME_UNIT < failedTimeInterval;
}
/**
* handling the list of tasks to be submitted
*/
private void submitStandByTask(){
for(Map.Entry<String, TaskInstance> entry: readyToSubmitTaskList.entrySet()) {
TaskInstance task = entry.getValue();
DependResult dependResult = getDependResultForTask(task);
if(DependResult.SUCCESS == dependResult){
if(retryTaskIntervalOverTime(task)){
submitTaskExec(task);
removeTaskFromStandbyList(task);
}
}else if(DependResult.FAILED == dependResult){
// if the dependency fails, the current node is not submitted and the state changes to failure.
dependFailedTask.put(entry.getKey(), task);
removeTaskFromStandbyList(task);
logger.info("task {},id:{} depend result : {}",task.getName(), task.getId(), dependResult);
}
}
}
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
/**
* get recovery task instance
* @param taskId task id
* @return recovery task instance
*/
private TaskInstance getRecoveryTaskInstance(String taskId){
if(!StringUtils.isNotEmpty(taskId)){
return null;
}
try {
Integer intId = Integer.valueOf(taskId);
Refactor Architecture Basic modification #1658 (#1946) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module (#1925) * 1,remove dolphinscheduler-rpc module 2,add dolphinscheduler-remote module 3,add dolphinscheduler-service module 4,refactor LoggerServer module * ProcessUtils modify * Refactor architecture (#1926) * move version to parent pom * move version properties to parent pom for easy management * remove freemarker dependency * delete CombinedApplicationServer * #1871 correct spelling * #1873 some updates for TaskQueueZkImpl * #1875 remove unused properties in pom * #1878 1. remove tomcat dependency 2. remove combined_logback.xml in api module 3. format pom.xml for not aligning * #1885 fix api server startup failure 1. add jsp-2.1 dependency 2. remove jasper-runtime dependency * add stringutils ut (#1921) * add stringutils ut * Newfeature for #1675. (#1908) Continue to finish the rest works, add the cache feature for dependence,mr,python,sub_process,procedure and shell. * Add modify user name for process definition (#1919) * class overrides equals() and should therefore also override hashCode() * #1862 add modify user in process difinition list * #1862 add pg-1.2.2 ddl.sql * modify ScriptRunnerTest * add updateProessDifinition UT * modify updateProcessDifinition UT * modify updateProcessDifinition UT * modify mysql 1.2.2 ddl.sql&dml.sql * add scope test to mysql in pom * modify pg-1.2.2 ddl.sql * refactor module * updates Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com> * dolphinscheduler-common remove spring (#1931) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * SpringApplicationContext class title add license (#1932) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * add license (#1934) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * Refactor architecture (#1936) * move datasource classes to dao module * fix send4LetterWord bug * LoggerServiceTest remove ProcessDao (#1944) * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * dolphinscheduler-common remove spring * LoggerServiceTest remove ProcessDao * exclude jasper-compiler in case of runtime conflict (#1938) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * DataAnaylysisServiceTest and ProcessDefinitionService modify * remote module add comment * OSUtilsTest modify * add finally block to close channel (#1951) * move datasource classes to dao module * fix send4LetterWord bug * exclude jasper-compiler in case of runtime conflict * add finally block to close channel Co-authored-by: Tboy <technoboy@yeah.net> Co-authored-by: khadgarmage <khadgar.mage@outlook.com> Co-authored-by: zhukai <boness@qq.com> Co-authored-by: Yelli <amarantine@my.com>
4 years ago
TaskInstance task = processService.findTaskInstanceById(intId);
if(task == null){
logger.error("start node id cannot be found: {}", taskId);
}else {
return task;
}
}catch (Exception e){
logger.error("get recovery task instance failed ",e);
}
return null;
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* get start task instance list
* @param cmdParam command param
* @return task instance list
*/
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
private List<TaskInstance> getStartTaskInstanceList(String cmdParam){
List<TaskInstance> instanceList = new ArrayList<>();
Map<String, String> paramMap = JSONUtils.toMap(cmdParam);
if(paramMap != null && paramMap.containsKey(CMDPARAM_RECOVERY_START_NODE_STRING)){
String[] idList = paramMap.get(CMDPARAM_RECOVERY_START_NODE_STRING).split(Constants.COMMA);
for(String nodeId : idList){
TaskInstance task = getRecoveryTaskInstance(nodeId);
if(task != null){
instanceList.add(task);
}
}
}
return instanceList;
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* parse "StartNodeNameList" from cmd param
* @param cmdParam command param
* @return start node name list
*/
private List<String> parseStartNodeName(String cmdParam){
List<String> startNodeNameList = new ArrayList<>();
Map<String, String> paramMap = JSONUtils.toMap(cmdParam);
if(paramMap == null){
return startNodeNameList;
}
if(paramMap.containsKey(CMDPARAM_START_NODE_NAMES)){
startNodeNameList = Arrays.asList(paramMap.get(CMDPARAM_START_NODE_NAMES).split(Constants.COMMA));
}
return startNodeNameList;
}
/**
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* generate start node name list from parsing command param;
* if "StartNodeIdList" exists in command param, return StartNodeIdList
* @return recovery node name list
*/
private List<String> getRecoveryNodeNameList(){
List<String> recoveryNodeNameList = new ArrayList<>();
if(CollectionUtils.isNotEmpty(recoverNodeIdList)) {
for (TaskInstance task : recoverNodeIdList) {
recoveryNodeNameList.add(task.getName());
}
}
return recoveryNodeNameList;
}
/**
* generate flow dag
Add method and parameters comments (#1220) * rename from DatasourceUserMapper to DataSourceUserMapper * add unit test in UserMapper and WorkerGroupMapper * change cn.escheduler to org.apache.dolphinscheduler * add unit test in UdfFuncMapperTest * add unit test in UdfFuncMapperTest * remove DatabaseConfiguration * add ConnectionFactoryTest * cal duration in processInstancesList * change desc to description * change table name in mysql ddl * change table name in mysql ddl * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * change escheduler to dolphinscheduler * remove log4j-1.2-api and modify AlertMapperTest * remove log4j-1.2-api * Add alertDao to spring management * Add alertDao to spring management * get SqlSessionFactory from MybatisSqlSessionFactoryBean * get processDao by DaoFactory * read druid properties in ConneciontFactory * read druid properties in ConneciontFactory * change get alertDao by spring to DaoFactory * add log4j to resolve #967 * resole verify udf name error and delete udf error * Determine if principal is empty * Determine whether the logon user has the right to delete the project * Fixed an issue that produced attatch file named such as ATT00002.bin * fix too many connection in upgrade or create * fix NEED_FAULT_TOLERANCE and WAITTING_THREAD count fail * Added a judgment on whether the currently login user is an administrator * fix update udf database not change and create time is changed * add enterprise.wechat.enable to decide whether to send enterprise WeChat * change method check * Remove the administrator's judgment on query access token list * only admin can create worker group * delete alert group need delete the relation of user and alert group * add timeout in proxy when upload large resource * add gets scheduled times by expect fire times * add gets scheduled times by expect fire times * Increase the judgment of whether it is admin * Increase the judgment of whether it is admin * when delete access token add whether login user has perm to delete * change mysql-connector-java scope to test * update scm test * add profile test * Add method and parameters comments * roll back
5 years ago
* @param processDefinitionJson process definition json
* @param startNodeNameList start node name list
* @param recoveryNodeNameList recovery node name list
* @param depNodeType depend node type
* @return ProcessDag process dag
* @throws Exception exception
*/
public ProcessDag generateFlowDag(String processDefinitionJson,
List<String> startNodeNameList,
List<String> recoveryNodeNameList,
TaskDependType depNodeType)throws Exception{
return DagHelper.generateFlowDag(processDefinitionJson, startNodeNameList, recoveryNodeNameList, depNodeType);
}
}